You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/11/14 00:51:07 UTC

[GitHub] [helix] kaisun2000 opened a new pull request #1532: Per replica throttle (WIP)

kaisun2000 opened a new pull request #1532:
URL: https://github.com/apache/helix/pull/1532


   ### Issues
   
   - [ ] My PR addresses the following Helix issues and references them in the PR description:
   
   # resolve #343 
   
   ### Description
   
   - [x] Here are some details about my PR, including screenshots of any UI changes:
   
     Per replica throttling replacing intermediate stage which is partition based. The finer granularity
     would skip boosting unnecessary replica in a recovery partition.
   
   ### Tests
   
   - [ ] The following tests are written for this issue:
   
   (List the names of added unit/integration tests)
   
   - [ ] The following is the result of the "mvn test" command on the appropriate module:
   
   (Before CI test pass, please copy & paste the result of "mvn test")
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555427942



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }

Review comment:
       The removing idea theoretically also works.  
   
   `MessageOutput` is the class holding the message. However, this class does not have any removing message helper functions. It is debatable whether we want to add message removing apis to this class.
   
   Considering the benefit of using alternative approach is not really essential, maybe let us just keep the existing approach?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831306



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestNoThrottleDisabledPartitions.java
##########
@@ -37,9 +38,11 @@
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.builder.FullAutoModeISBuilder;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.jboss.netty.handler.codec.http.websocketx.WebSocketServerProtocolHandshakeHandler;

Review comment:
       What's this?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r554258542



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       This is a good suggestion to make the code more generic. The accumulation logic is added in `getPartitionExpectedAndCurrentStateCountMap` and the testing of load/recovery logic is simplified a lot here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541287517



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,

Review comment:
       Any proposal? I can adopt the name you proposed.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546002850



##########
File path: helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
##########
@@ -45,6 +47,8 @@
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import sun.applet.Main;

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on pull request #1532:
URL: https://github.com/apache/helix/pull/1532#issuecomment-743448796


   https://github.com/apache/helix/pull/1532#discussion_r541205021
   
   > This method seems to be added for debug log only.
   For simplifying the business logic, I suggest don't add this method but relying on the existing get method to retrieve information
   
   This is not for debug info only. Note the selectedMessage from previous stage is of this type. Before, we don't need to access it this way. Now per replica stage is using MessageOutput as input, this why we need this method.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541457195



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,

Review comment:
       throttlePerReplicaMessages? 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541481650



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;
+    }
+    return Collections.emptyMap();
+  }
+
+  public void addResourceMessages(String resourceName, Map<Partition, List<Message>> resourceMessages) {

Review comment:
       changed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546993916



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);

Review comment:
       nit:
   retracedStateMap.put(message.getTgtName(), message.getToState());
   might be cleaner. 
   

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);

Review comment:
       nit:
   ```
   retracedStateMap.put(message.getTgtName(), message.getToState());
   ```
   might be cleaner. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r545335228



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       This logic and even comment are exactly the same as before. As you know, this is intentional to preserve them here, thus it would avoid any potential change of logic in production. 
   
   Let me add a TODO here




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831172



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,775 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.rmi.runtime.Log;

Review comment:
       Why are we using this library?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555366719



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }

Review comment:
       removed duplicated code.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541369193



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,

Review comment:
       Renamed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533104685



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,775 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.tools.ClusterVerifiers.StrictMatchExternalViewVerifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.rmi.runtime.Log;

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r570636926



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/StateTransitionThrottleController.java
##########
@@ -46,6 +46,20 @@
 
   private boolean _throttleEnabled = false;
 
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StateTransitionThrottleController _pendingTransitionAllowedInCluster:");
+    sb.append(_pendingTransitionAllowedInCluster.toString());
+    sb.append("\n");
+    sb.append("_pendingTransitionAllowedPerInstance:");
+    sb.append(_pendingTransitionAllowedPerInstance.toString());
+    sb.append("\n");
+    sb.append("_pendingTransitionAllowedPerResource:");
+    sb.append(_pendingTransitionAllowedPerResource);
+    sb.append("\n");
+    return sb.toString();
+  }

Review comment:
       ```suggestion
       sb.append(_pendingTransitionAllowedInCluster.toString());
       sb.append("\n");
       sb.append("_pendingTransitionAllowedPerInstance:");
       sb.append(_pendingTransitionAllowedPerInstance.toString());
       sb.append("\n");
       sb.append("_pendingTransitionAllowedPerResource:");
       sb.append(_pendingTransitionAllowedPerResource);
       sb.append("\n");
       return sb.toString();
     }
   ```
   
   asdfsdfasdfsdafasdf




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533778638



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -483,7 +483,7 @@ private void chargePendingTransition(Resource resource, CurrentStateOutput curre
         rebalanceType = StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE;
       }
 
-      if (pendingMap.size() > 0) {
+      if (pendingMap.size() > 0 && rebalanceType != RebalanceType.NONE) {

Review comment:
       This could be meaningless since we will remove the whole stage anyway.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;

Review comment:
       Shall we return Collections.unmodifiablemap?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THOTTLED_LOAD_MESSAGES

Review comment:
       Why we need this config? As we said, we should keep backward compatible with old throttling configs.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,951 @@
+package org.apache.helix.controller.stages;

Review comment:
       This new file is hard to review... Can we do this? Since most of the code is copied from IntermediateStage. Let's use IntermediateStage file and make the change for the review first as a PR? Then after reviewed, we can rename the file to PerReplicaThrottleStage.
   
   I know split it into two PRs could break tests for first one. But I think that could be more efficient for reviewing.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541477033



##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -481,6 +482,7 @@ private static PipelineRegistry createDefaultRegistry(String pipelineName) {
       rebalancePipeline.addStage(new MaintenanceRecoveryStage());

Review comment:
       moved MaintenanceRecoveryStage after Per replica stages and also changed corresponding input change to MaintenanceRecoveryStage




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831315



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestNoThrottleDisabledPartitions.java
##########
@@ -81,7 +84,7 @@ public void testDisablingTopStateReplicaByDisablingInstance() throws Exception {
     ClusterControllerManager controller =
         new ClusterControllerManager(ZK_ADDR, _clusterName, "controller_0");
     controller.syncStart();
-    Thread.sleep(500L);
+    Thread.sleep(10000L);

Review comment:
       We probably shouldn't do this...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555386744



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541286820



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       Thought about it before. This worth another diff. This one is already very large.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       will add a to-do.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r543886602



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_OUTPUT_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THOTTLED_LOAD_MESSAGES

Review comment:
       Typo: `PER_REPLICA_THOTTLED_LOAD_MESSAGES` -> `PER_REPLICA_THROTTLED_LOAD_MESSAGES`

##########
File path: helix-core/src/test/java/org/apache/helix/mock/participant/MockOFModelFactory.java
##########
@@ -0,0 +1,53 @@
+package org.apache.helix.mock.participant;
+/*
+ * 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.participant.statemachine.StateModelFactory;
+
+// mock online offline state model factory
+public class MockOFModelFactory extends StateModelFactory<MockOFStateModel> {
+  private MockTransition _transition;
+
+  public MockOFModelFactory() {
+    this(null);
+  }
+
+  public MockOFModelFactory(MockTransition transition) {
+    _transition = transition;
+  }
+
+  public void setTrasition(MockTransition transition) {

Review comment:
       Typo. `setTrasition` -> `setTransition`

##########
File path: helix-core/src/test/java/org/apache/helix/mock/MockHelixAdmin.java
##########
@@ -45,6 +47,8 @@
 import org.apache.helix.model.ResourceConfig;
 import org.apache.helix.model.StateModelDefinition;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import sun.applet.Main;

Review comment:
       Remove the unused imports  in this file/PR?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/resource/ResourceMessageGenerationPhase.java
##########
@@ -19,7 +19,9 @@
  * under the License.
  */
 
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.BestPossibleStateOutput;

Review comment:
       This import is unused.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546895772



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       Checked. This is not a concern. Indeed both chargingPendingMessage and later classifyMessages all use the same sorting comparator.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r547557367



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       No need to check this. I know message non-upwards are load but anyway we need to check expected state vs current state. This check is not necessary.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }

Review comment:
       This code logic is very duplicated. We can refactor into previous message type decision piece. This would make the code much concise.
   
   loop message:
      RebalanceType type = xxxx;
       throttleController
               .chargeInstance(type,
                   toInstance);
           throttleController
               .chargeCluster(type);
           throttleController
               .chargeResource(type,
                   resourceName);
           logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       A general way is we have 2 things:
   1. As you implemented the expected state map.
   2. Totally matched replica numbers. This number is counting from top priority of the state and accumulatively down to the number of replicas. But there is another rule of "R".
   
   I would suggest you first try to understand the state model rule of how we counting different states of different replicas first.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       Same as I suggested above, let's loop the priority state with an accumulative number for it.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+      ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {
+      logger.error("onlyDownwardLoadBalance can't be used together with recovery_rebalance");
+      return;
+    }
+
+    String resourceName = resource.getResourceName();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    messages.sort(new MessageThrottleComparator(bestPossibleMap, currentStateMap, messagePartitionMap, stateModelDef,isRecovery));
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg: messages) {
+      if (onlyDownwardLoadBalance) {
+        boolean isDownward = isDownwardTransition(idealState, cache, msg);
+        if (isDownward == false) {

Review comment:
       NIT: no other usage of isDownward. Let's just have it as:
   
   if (isDownwardTransition(idealState, cache, msg)) {
   }




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r534665820



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
     }
 
-    IntermediateStateOutput intermediateStateOutput =
-        compute(event, resourceToRebalance, currentStateOutput, bestPossibleStateOutput);
-    event.addAttribute(AttributeName.INTERMEDIATE_STATE.name(), intermediateStateOutput);
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
 
-    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
-    // it does, pause the rebalance and put the cluster on maintenance mode
-    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
-    if (maxPartitionPerInstance > 0) {
-      validateMaxPartitionsPerInstance(event, cache, intermediateStateOutput,
-          maxPartitionPerInstance);
-    }
-  }
-
-  /**
-   * Go through each resource, and based on BestPossibleState and CurrentState, compute
-   * IntermediateState as close to BestPossibleState while maintaining throttling constraints (for
-   * example, ensure that the number of possible pending state transitions does NOT go over the set
-   * threshold).
-   * @param event
-   * @param resourceMap
-   * @param currentStateOutput
-   * @param bestPossibleStateOutput
-   * @return
-   */
-  private IntermediateStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput) {
-    IntermediateStateOutput output = new IntermediateStateOutput();
-    ResourceControllerDataProvider dataCache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
-
-    StateTransitionThrottleController throttleController = new StateTransitionThrottleController(
-        resourceMap.keySet(), dataCache.getClusterConfig(), dataCache.getLiveInstances().keySet());
-
-    // Resource level prioritization based on the numerical (sortable) priority field.
-    // If the resource priority field is null/not set, the resource will be treated as lowest
-    // priority.
-    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
-    for (String resourceName : resourceMap.keySet()) {
-      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
-    }
-    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
-    // in priority by keeping all priorities at MIN_VALUE
-    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
-      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
-      for (ResourcePriority resourcePriority : prioritizedResourceList) {
-        String resourceName = resourcePriority.getResourceName();
-
-        // Will take the priority from ResourceConfig first
-        // If ResourceConfig does not exist or does not have this field.
-        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
-        if (dataCache.getResourceConfig(resourceName) != null
-            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
-        } else if (dataCache.getIdealState(resourceName) != null && dataCache
-            .getIdealState(resourceName).getRecord().getSimpleField(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
         }
       }
-      prioritizedResourceList.sort(new ResourcePriorityComparator());
     }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
 
-    ClusterStatusMonitor clusterStatusMonitor =
-        event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    List<String> failedResources = new ArrayList<>();
-
-    // Priority is applied in assignment computation because higher priority by looping in order of
-    // decreasing priority
-    for (ResourcePriority resourcePriority : prioritizedResourceList) {
-      String resourceName = resourcePriority.getResourceName();
-
-      if (!bestPossibleStateOutput.containsResource(resourceName)) {
-        LogUtil.logInfo(logger, _eventId, String.format(
-            "Skip calculating intermediate state for resource %s because the best possible state is not available.",
-            resourceName));
-        continue;
-      }
-
-      Resource resource = resourceMap.get(resourceName);
-      IdealState idealState = dataCache.getIdealState(resourceName);
-      if (idealState == null) {
-        // If IdealState is null, use an empty one
-        LogUtil.logInfo(logger, _eventId,
-            String.format(
-                "IdealState for resource %s does not exist; resource may not exist anymore",
-                resourceName));
-        idealState = new IdealState(resourceName);
-        idealState.setStateModelDefRef(resource.getStateModelDefRef());
-      }
-
-      try {
-        output.setState(resourceName,
-            computeIntermediatePartitionState(dataCache, clusterStatusMonitor, idealState,
-                resourceMap.get(resourceName), currentStateOutput,
-                bestPossibleStateOutput.getPartitionStateMap(resourceName),
-                bestPossibleStateOutput.getPreferenceLists(resourceName), throttleController));
-      } catch (HelixException ex) {
-        LogUtil.logInfo(logger, _eventId,
-            "Failed to calculate intermediate partition states for resource " + resourceName, ex);
-        failedResources.add(resourceName);
-      }
+    if (isEmitThrottledMsg) {

Review comment:
       Why we need this? Is that for backtrace the intermediate state map?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831033



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -57,6 +57,8 @@
 import org.apache.helix.util.HelixUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import sun.rmi.runtime.Log;

Review comment:
       Please remove this.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482955



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -259,14 +181,7 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
                       + " mode due to an instance being assigned more replicas/partitions than "
                       + "the limit.");
             }
-
-            ClusterStatusMonitor clusterStatusMonitor =
-                event.getAttribute(AttributeName.clusterStatusMonitor.name());
-            if (clusterStatusMonitor != null) {
-              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
-                  ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED);
-            }
-            // Throw an exception here so that messages won't be sent out based on this mapping
+            //TODO: add metrics

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523830984



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
##########
@@ -108,4 +108,12 @@ public void setPreferenceLists(String resource,
   protected boolean containsResource(String resource) {
     return _preferenceLists != null && _preferenceLists.containsKey(resource);
   }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("preference list= ").append(_preferenceLists);
+    sb.append(", resourceStateMap= ").append(_resourceStateMap);
+    return sb.toString();
+  }

Review comment:
       Are we printing/logging this? If so, I don't think that's wise since that would be a lot of text being logged.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r545994437



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
     }
 
-    IntermediateStateOutput intermediateStateOutput =
-        compute(event, resourceToRebalance, currentStateOutput, bestPossibleStateOutput);
-    event.addAttribute(AttributeName.INTERMEDIATE_STATE.name(), intermediateStateOutput);
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
 
-    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
-    // it does, pause the rebalance and put the cluster on maintenance mode
-    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
-    if (maxPartitionPerInstance > 0) {
-      validateMaxPartitionsPerInstance(event, cache, intermediateStateOutput,
-          maxPartitionPerInstance);
-    }
-  }
-
-  /**
-   * Go through each resource, and based on BestPossibleState and CurrentState, compute
-   * IntermediateState as close to BestPossibleState while maintaining throttling constraints (for
-   * example, ensure that the number of possible pending state transitions does NOT go over the set
-   * threshold).
-   * @param event
-   * @param resourceMap
-   * @param currentStateOutput
-   * @param bestPossibleStateOutput
-   * @return
-   */
-  private IntermediateStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput) {
-    IntermediateStateOutput output = new IntermediateStateOutput();
-    ResourceControllerDataProvider dataCache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
-
-    StateTransitionThrottleController throttleController = new StateTransitionThrottleController(
-        resourceMap.keySet(), dataCache.getClusterConfig(), dataCache.getLiveInstances().keySet());
-
-    // Resource level prioritization based on the numerical (sortable) priority field.
-    // If the resource priority field is null/not set, the resource will be treated as lowest
-    // priority.
-    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
-    for (String resourceName : resourceMap.keySet()) {
-      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
-    }
-    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
-    // in priority by keeping all priorities at MIN_VALUE
-    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
-      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
-      for (ResourcePriority resourcePriority : prioritizedResourceList) {
-        String resourceName = resourcePriority.getResourceName();
-
-        // Will take the priority from ResourceConfig first
-        // If ResourceConfig does not exist or does not have this field.
-        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
-        if (dataCache.getResourceConfig(resourceName) != null
-            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
-        } else if (dataCache.getIdealState(resourceName) != null && dataCache
-            .getIdealState(resourceName).getRecord().getSimpleField(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
         }
       }
-      prioritizedResourceList.sort(new ResourcePriorityComparator());
     }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
 
-    ClusterStatusMonitor clusterStatusMonitor =
-        event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    List<String> failedResources = new ArrayList<>();
-
-    // Priority is applied in assignment computation because higher priority by looping in order of
-    // decreasing priority
-    for (ResourcePriority resourcePriority : prioritizedResourceList) {
-      String resourceName = resourcePriority.getResourceName();
-
-      if (!bestPossibleStateOutput.containsResource(resourceName)) {
-        LogUtil.logInfo(logger, _eventId, String.format(
-            "Skip calculating intermediate state for resource %s because the best possible state is not available.",
-            resourceName));
-        continue;
-      }
-
-      Resource resource = resourceMap.get(resourceName);
-      IdealState idealState = dataCache.getIdealState(resourceName);
-      if (idealState == null) {
-        // If IdealState is null, use an empty one
-        LogUtil.logInfo(logger, _eventId,
-            String.format(
-                "IdealState for resource %s does not exist; resource may not exist anymore",
-                resourceName));
-        idealState = new IdealState(resourceName);
-        idealState.setStateModelDefRef(resource.getStateModelDefRef());
-      }
-
-      try {
-        output.setState(resourceName,
-            computeIntermediatePartitionState(dataCache, clusterStatusMonitor, idealState,
-                resourceMap.get(resourceName), currentStateOutput,
-                bestPossibleStateOutput.getPartitionStateMap(resourceName),
-                bestPossibleStateOutput.getPreferenceLists(resourceName), throttleController));
-      } catch (HelixException ex) {
-        LogUtil.logInfo(logger, _eventId,
-            "Failed to calculate intermediate partition states for resource " + resourceName, ex);
-        failedResources.add(resourceName);
-      }
+    if (isEmitThrottledMsg) {

Review comment:
       This is for testing purpose. The constructor in the other PerReplicaStage is changed to protected as we usually do. See the comment with JJ.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546003756



##########
File path: helix-core/src/test/java/org/apache/helix/mock/participant/MockOFModelFactory.java
##########
@@ -0,0 +1,53 @@
+package org.apache.helix.mock.participant;
+/*
+ * 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.participant.statemachine.StateModelFactory;
+
+// mock online offline state model factory
+public class MockOFModelFactory extends StateModelFactory<MockOFStateModel> {
+  private MockTransition _transition;
+
+  public MockOFModelFactory() {
+    this(null);
+  }
+
+  public MockOFModelFactory(MockTransition transition) {
+    _transition = transition;
+  }
+
+  public void setTrasition(MockTransition transition) {

Review comment:
       changed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482632



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;

Review comment:
       Feel free to skip the IntermediateStateCalcStage. Previously JK asked to compare IntermediateStateCalcStage with PerReplicaThrottleStage as it is deemed easier to look at the difference. But the new code can be done in a way to change just a few lines from IntermediateStateCalcStage as there new code is written from scratch with different input and output even the logic is similar




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541413606



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {

Review comment:
       Good point. made this one protected.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533106741



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
##########
@@ -108,4 +108,12 @@ public void setPreferenceLists(String resource,
   protected boolean containsResource(String resource) {
     return _preferenceLists != null && _preferenceLists.containsKey(resource);
   }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("preference list= ").append(_preferenceLists);
+    sb.append(", resourceStateMap= ").append(_resourceStateMap);
+    return sb.toString();
+  }

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on pull request #1532:
URL: https://github.com/apache/helix/pull/1532#issuecomment-728275075


   @narendly, thx for the early review. Still in early stage for this feature.Ssome major refactoring and adding more tests are pending.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r554259044



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       Basically `PartitionMessageComparator` is only used to sort message (replica) with a partition. This logic is not need in previous IntermediateCalStage. 
   `MessageThrottleComparator` is use to determine the message order across partitions and this is the same logic as PartitionPriorityComparator in IntermediateCalcStage.
   
   What is you take?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533104914



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,5 +48,8 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  PER_REPLICA_THROTTLED_MESSAGES,
+  PER_REPLICA_RETRACED_STATES

Review comment:
       comment added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533107438



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestNoThrottleDisabledPartitions.java
##########
@@ -81,7 +84,7 @@ public void testDisablingTopStateReplicaByDisablingInstance() throws Exception {
     ClusterControllerManager controller =
         new ClusterControllerManager(ZK_ADDR, _clusterName, "controller_0");
     controller.syncStart();
-    Thread.sleep(500L);
+    Thread.sleep(10000L);

Review comment:
       restored.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on pull request #1532:
URL: https://github.com/apache/helix/pull/1532#issuecomment-834664881


   Close due to inactive.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r553625904



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       Maybe there is some confusion here. Let me first make sure we are on the same page about usage of two different comparator -- PartitionMessageComparator and MessageThrottleComparator.
   
   `MessageThrottleComparator` is the counter part of `PartitionPriorityComparator` in IntermediateCalcStage. 
   
   1/ `PartitionMessageComparator` are used in two places, namely a) pendingMessages load/recovery classification and b) input message from previous stage load/recovery classification.  -- note in the previous IntermediateCalcStage, the is not need for such a comparator as the load/recovery classification is done per partition (not per replica), thus there is no need to give a order.
   
   2/ `MessageThrottleComparator` is basically the same as `PartitionPriorityComparator` in IntermediateCalcStage. The usage is to give an order when throttling is applied to load/recovery messages.
   The logic is the following -- note this is basically the same as `PartitionPriorityComparator` in IntermediateCalcStage
   - Higher priority for topState
   - Higher priority for the partition with fewer active replicas
   - Higher priority for the partition with fewer replicas with states matching with IdealState




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r570636751



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_OUTPUT_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THROTTLED_LOAD_MESSAGES

Review comment:
       ```suggestion
     TO_BE_PURGED_JOBS_MAP,
   
     // This attribute denotes the messages output from Per Preplica Throttle stage
     PER_REPLICA_OUTPUT_MESSAGES,
   
     // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
     PER_REPLICA_RETRACED_STATES,
   
     // This attribute denotes the filtered out messages deemed as recovery message
     PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
   
     // This attribute denotes the filtered out messages deemed as load message
     PER_REPLICA_THROTTLED_LOAD_MESSAGES
   ```

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/StateTransitionThrottleController.java
##########
@@ -46,6 +46,20 @@
 
   private boolean _throttleEnabled = false;
 
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("StateTransitionThrottleController _pendingTransitionAllowedInCluster:");
+    sb.append(_pendingTransitionAllowedInCluster.toString());
+    sb.append("\n");
+    sb.append("_pendingTransitionAllowedPerInstance:");
+    sb.append(_pendingTransitionAllowedPerInstance.toString());
+    sb.append("\n");
+    sb.append("_pendingTransitionAllowedPerResource:");
+    sb.append(_pendingTransitionAllowedPerResource);
+    sb.append("\n");
+    return sb.toString();
+  }

Review comment:
       ```suggestion
       sb.append(_pendingTransitionAllowedInCluster.toString());
       sb.append("\n");
       sb.append("_pendingTransitionAllowedPerInstance:");
       sb.append(_pendingTransitionAllowedPerInstance.toString());
       sb.append("\n");
       sb.append("_pendingTransitionAllowedPerResource:");
       sb.append(_pendingTransitionAllowedPerResource);
       sb.append("\n");
       return sb.toString();
     }
   ```
   
   asdfsdfasdfsdafasdf




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546004066



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/resource/ResourceMessageGenerationPhase.java
##########
@@ -19,7 +19,9 @@
  * under the License.
  */
 
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.stages.AttributeName;
+import org.apache.helix.controller.stages.BestPossibleStateOutput;

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r534492347



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -483,7 +483,7 @@ private void chargePendingTransition(Resource resource, CurrentStateOutput curre
         rebalanceType = StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE;
       }
 
-      if (pendingMap.size() > 0) {
+      if (pendingMap.size() > 0 && rebalanceType != RebalanceType.NONE) {

Review comment:
       Agree. But this is a bug previously we have. It is also in production now. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541456147



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/StateTransitionThrottleController.java
##########
@@ -46,6 +46,20 @@
 
   private boolean _throttleEnabled = false;
 
+  @Override
+  public String toString() {

Review comment:
       Got it. I think it would be helpful.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555428564



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+      ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {
+      logger.error("onlyDownwardLoadBalance can't be used together with recovery_rebalance");
+      return;
+    }
+
+    String resourceName = resource.getResourceName();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    messages.sort(new MessageThrottleComparator(bestPossibleMap, currentStateMap, messagePartitionMap, stateModelDef,isRecovery));
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg: messages) {
+      if (onlyDownwardLoadBalance) {
+        boolean isDownward = isDownwardTransition(idealState, cache, msg);
+        if (isDownward == false) {
+          throttledMessages.add(msg);
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s throttled in resource as not downward: %s with type: %s", msg, resourceName,
+                    rebalanceType));
+          }
+          continue;
+        }
+      }
+
+      if (throttleController.shouldThrottleForResource(rebalanceType, resourceName)) {
+        throttledMessages.add(msg);
+        if (logger.isDebugEnabled()) {
+          LogUtil.logDebug(logger, _eventId,
+              String.format("Message: %s throttled in resource: %s with type: %s", msg, resourceName,
+                  rebalanceType));
+        }
+        continue;
+      }
+      String instance = msg.getTgtName();
+      if (throttleController.shouldThrottleForInstance(rebalanceType,instance)) {
+        throttledMessages.add(msg);
+        if (logger.isDebugEnabled()) {
+          LogUtil.logDebug(logger, _eventId,
+              String.format("Message: %s throttled in instance %s in resource: %s with type: %s", instance, msg, resourceName,
+                  rebalanceType));
+        }
+        continue;
+      }
+      throttleController.chargeInstance(rebalanceType, instance);
+      throttleController.chargeResource(rebalanceType, resourceName);
+      throttleController.chargeCluster(rebalanceType);
+      logger.trace("throttleControllerstate->{} after charge load msg: {}", throttleController, msg);
+    }
+  }
+
+   // ------------------ utilities ---------------------------
+  /**
+   * POJO that maps resource name to its priority represented by an integer.
+   */
+  private static class ResourcePriority {
+    private String _resourceName;
+    private int _priority;
+
+    ResourcePriority(String resourceName, Integer priority) {
+      _resourceName = resourceName;
+      _priority = priority;
+    }
+
+    public int compareTo(ResourcePriority resourcePriority) {
+      return Integer.compare(_priority, resourcePriority._priority);
+    }
+
+    public String getResourceName() {
+      return _resourceName;
+    }
+
+    public void setPriority(String priority) {
+      try {
+        _priority = Integer.parseInt(priority);
+      } catch (Exception e) {
+        logger.warn(
+            String.format("Invalid priority field %s for resource %s", priority, _resourceName));
+      }
+    }
+  }
+
+  private static class ResourcePriorityComparator implements Comparator<ResourcePriority> {
+    @Override
+    public int compare(ResourcePriority priority1, ResourcePriority priority2) {
+      return priority2.compareTo(priority1);
+    }
+  }
+
+  // compare message for throttling, note, all these message are of type state_transition: how about upward, downward?
+  // recovery are all upward
+  // 1) toState priority (toTop is higher than toSecond)
+  // 2) same toState, the message classification time, the less required toState meeting minActive requirement has higher priority
+  // 3) Higher priority for the partition of messages with fewer replicas with states matching with bestPossible ??? do we need this one
+  private static class MessageThrottleComparator implements Comparator<Message> {

Review comment:
       See my comment before. Let us chat about this one.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831020



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -81,11 +83,15 @@ public void process(ClusterEvent event) throws Exception {
       throw new StageException(
           "Missing attributes in event:" + event + ". Requires CURRENT_STATE|RESOURCES|DataCache");
     }
+    
+    // LogUtil.logInfo(logger, _eventId, String.format("CurrentState for bestpossible is: %s", currentStateOutput));

Review comment:
       Please clean up these comments.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r545997352



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {

Review comment:
       Previously we have done this before, but it does not help. The main thing is that the input and output is different between these two stages. Thus the not 1:1 method correspondence.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546005159



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_OUTPUT_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THOTTLED_LOAD_MESSAGES

Review comment:
       fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r544118883



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       TODO is totally fine, but we might need to check in this change into a branch first. Otherwise, it will break the already existing metrics in the master.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       DEFAULT_ERROR_OR_RECOVERY_PARTITION_THRESHOLD_FOR_LOAD_BALANCE = -1 and we need the logic here to translate it to 1. What a mess.
   Can we please refine this logic since you are already on top of it?
   
   I assume it is a simple change. But if not, we can just add a TODO here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541373391



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();

Review comment:
       addressed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546984789



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       I think it could be simplified into 
   if....else if ....else.
   And we don't need the nested if here. 
   
   Also, maybe better naming for `partitionCount `.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555429183



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());

Review comment:
       PreferenceList is null seem to be in the case of dropping instances, let me test this a little bit more to make sure this part is addressed carefully. This can be potentially an issue. Will update a little bit later.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r552971248



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       @xyuanlu. This can be further simplified. Currently we keep the old code logic just to make sure there is not behavior change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541455433



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;
+    }
+    return Collections.emptyMap();
+  }
+
+  public void addResourceMessages(String resourceName, Map<Partition, List<Message>> resourceMessages) {

Review comment:
       I understand your concern. One easy way to avoid this performance concern, while avoiding breaking the clean API design, is pass the output into the computePerReplicaPartitionState() methods. And inside the method, update the messages gradually use the existing API.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541351056



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,

Review comment:
       This is valid point. Later I also regret to use this name. Let me change to PER_REPLICA_OUTPUT_MESSAGE.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541426905



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/StateTransitionThrottleController.java
##########
@@ -46,6 +46,20 @@
 
   private boolean _throttleEnabled = false;
 
+  @Override
+  public String toString() {

Review comment:
       for example `applyThrottling()` use it here as ` logger.trace("throttleControllerstate->{} before load", throttleController);`
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541282599



##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -481,6 +482,7 @@ private static PipelineRegistry createDefaultRegistry(String pipelineName) {
       rebalancePipeline.addStage(new MaintenanceRecoveryStage());

Review comment:
       Good point, let me check the details.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541399550



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,

Review comment:
       changed to "computeReourcePartitionState()"




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555402867



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);

Review comment:
       Changes done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541371101



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;

Review comment:
       changed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] pkuwm commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
pkuwm commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r545343941



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {

Review comment:
       @kaisun2000 Maybe submit a diff between `PerReplicaThrottleStage` and `IntermediateStage` can help reviewers understand the logic change based on IntermediateStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546019612



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       Let me add in the implementation of metrics here too. This should be a small change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555390283



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {

Review comment:
       changed to outMessagesByPartition




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r553610577



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();

Review comment:
       This is for testing purpose. The test would assert the messages got throttled as load or recovery. It is not for printing purpose. See TestPerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541477700



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       to-do is added in `public void process(ClusterEvent event)` last line.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541233966



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
     }
 
-    IntermediateStateOutput intermediateStateOutput =
-        compute(event, resourceToRebalance, currentStateOutput, bestPossibleStateOutput);
-    event.addAttribute(AttributeName.INTERMEDIATE_STATE.name(), intermediateStateOutput);
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
 
-    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
-    // it does, pause the rebalance and put the cluster on maintenance mode
-    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
-    if (maxPartitionPerInstance > 0) {
-      validateMaxPartitionsPerInstance(event, cache, intermediateStateOutput,
-          maxPartitionPerInstance);
-    }
-  }
-
-  /**
-   * Go through each resource, and based on BestPossibleState and CurrentState, compute
-   * IntermediateState as close to BestPossibleState while maintaining throttling constraints (for
-   * example, ensure that the number of possible pending state transitions does NOT go over the set
-   * threshold).
-   * @param event
-   * @param resourceMap
-   * @param currentStateOutput
-   * @param bestPossibleStateOutput
-   * @return
-   */
-  private IntermediateStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput) {
-    IntermediateStateOutput output = new IntermediateStateOutput();
-    ResourceControllerDataProvider dataCache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
-
-    StateTransitionThrottleController throttleController = new StateTransitionThrottleController(
-        resourceMap.keySet(), dataCache.getClusterConfig(), dataCache.getLiveInstances().keySet());
-
-    // Resource level prioritization based on the numerical (sortable) priority field.
-    // If the resource priority field is null/not set, the resource will be treated as lowest
-    // priority.
-    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
-    for (String resourceName : resourceMap.keySet()) {
-      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
-    }
-    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
-    // in priority by keeping all priorities at MIN_VALUE
-    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
-      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
-      for (ResourcePriority resourcePriority : prioritizedResourceList) {
-        String resourceName = resourcePriority.getResourceName();
-
-        // Will take the priority from ResourceConfig first
-        // If ResourceConfig does not exist or does not have this field.
-        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
-        if (dataCache.getResourceConfig(resourceName) != null
-            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
-        } else if (dataCache.getIdealState(resourceName) != null && dataCache
-            .getIdealState(resourceName).getRecord().getSimpleField(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
         }
       }
-      prioritizedResourceList.sort(new ResourcePriorityComparator());
     }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
 
-    ClusterStatusMonitor clusterStatusMonitor =
-        event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    List<String> failedResources = new ArrayList<>();
-
-    // Priority is applied in assignment computation because higher priority by looping in order of
-    // decreasing priority
-    for (ResourcePriority resourcePriority : prioritizedResourceList) {
-      String resourceName = resourcePriority.getResourceName();
-
-      if (!bestPossibleStateOutput.containsResource(resourceName)) {
-        LogUtil.logInfo(logger, _eventId, String.format(
-            "Skip calculating intermediate state for resource %s because the best possible state is not available.",
-            resourceName));
-        continue;
-      }
-
-      Resource resource = resourceMap.get(resourceName);
-      IdealState idealState = dataCache.getIdealState(resourceName);
-      if (idealState == null) {
-        // If IdealState is null, use an empty one
-        LogUtil.logInfo(logger, _eventId,
-            String.format(
-                "IdealState for resource %s does not exist; resource may not exist anymore",
-                resourceName));
-        idealState = new IdealState(resourceName);
-        idealState.setStateModelDefRef(resource.getStateModelDefRef());
-      }
-
-      try {
-        output.setState(resourceName,
-            computeIntermediatePartitionState(dataCache, clusterStatusMonitor, idealState,
-                resourceMap.get(resourceName), currentStateOutput,
-                bestPossibleStateOutput.getPartitionStateMap(resourceName),
-                bestPossibleStateOutput.getPreferenceLists(resourceName), throttleController));
-      } catch (HelixException ex) {
-        LogUtil.logInfo(logger, _eventId,
-            "Failed to calculate intermediate partition states for resource " + resourceName, ex);
-        failedResources.add(resourceName);
-      }
+    if (isEmitThrottledMsg) {

Review comment:
       The would expose throttled messages out if configured. (If not configured, no effect). This would facilitate debugging and testing. Especially stage testing. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541228105



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THOTTLED_LOAD_MESSAGES

Review comment:
       Don't worry. These are not throttle config setting changes. Nothing changed in that front. This is the output and input name of the Per Replica Stages.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546877012



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }

Review comment:
       We dont need this. Usually we print out the message has been throttled. Next stage should have the message send out. It will be duplicated logs.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }

Review comment:
       Let's minimize code structure change. We can let the print happening in the code instead of carrying them out at top level and print all.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();

Review comment:
       No need to have these inputs and carrying them out.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -483,7 +483,7 @@ private void chargePendingTransition(Resource resource, CurrentStateOutput curre
         rebalanceType = StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE;
       }
 
-      if (pendingMap.size() > 0) {
+      if (pendingMap.size() > 0 && rebalanceType != RebalanceType.NONE) {

Review comment:
       Then this should be reflect to the new file instead of this file. We need to only remove it from the pipeline but also the file itself.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }

Review comment:
       We dont need to do some loops to construct. Why not we add all the message in new output. Then:
   1. If it is not FULL_AUTO, we directly return.
   2. If message has been throttled, we remove the message from output where it throttled.
   
   So we will not see such a block to construct result.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,

Review comment:
       Let's not change the order of the function. That's the reason diff can not be generated successfully.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r547561784



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       As I commented, the logic is complicated. Please merge them first. Otherwise, I cannot determine what is the difference. Will be very hard for reader understanding it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533104337



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestNoThrottleDisabledPartitions.java
##########
@@ -37,9 +38,11 @@
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.InstanceConfig;
 import org.apache.helix.model.LiveInstance;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.builder.FullAutoModeISBuilder;
 import org.apache.helix.tools.ClusterVerifiers.BestPossibleExternalViewVerifier;
 import org.apache.helix.zookeeper.datamodel.ZNRecord;
+import org.jboss.netty.handler.codec.http.websocketx.WebSocketServerProtocolHandshakeHandler;

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541201663



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -58,6 +58,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+

Review comment:
       remove?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -481,6 +482,7 @@ private static PipelineRegistry createDefaultRegistry(String pipelineName) {
       rebalancePipeline.addStage(new MaintenanceRecoveryStage());

Review comment:
       Note the comment in the previous line, will the new logic generate different output with the IntermediateStateCalcStage result, so we entering the maintenance mode in the wrong condition?
   Maybe we shall move the MaintenanceRecoveryStage after PerReplicaThrottleStage?

##########
File path: helix-core/src/test/resources/log4j.properties
##########
@@ -37,4 +39,8 @@ log4j.logger.org.apache=ERROR
 log4j.logger.com.noelios=ERROR
 log4j.logger.org.restlet=ERROR
 
+#log4j.logger.org.apache.helix.controller.stages.PerReplicaThrottleStage=INFO
+#log4j.logger.org.apache.helix.controller.stages.BestPossibleStateCalcStage=INFO
+
+#log4j.logger.org.apache.helix.integration.TestPerReplicaThrottle=INFO

Review comment:
       I assume these are for debugging? Please remove them for the official PR review.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();

Review comment:
       nit, dataCache.getClusterConfig().getResourcePriorityField() is called 2 times. We can use a local var to avoid one get.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {

Review comment:
       Or another option is to provide a protected set method so only the test can use it.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;
+    }
+    return Collections.emptyMap();
+  }
+
+  public void addResourceMessages(String resourceName, Map<Partition, List<Message>> resourceMessages) {

Review comment:
       I think the existing method is safer. Here if the caller modify the input map after this call, the output will be changed.
   Ideally, we shall only call "void addMessage(String resourceName, Partition partition, Message message)".
   Or modify the other set methods to use this basic one.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
##########
@@ -108,4 +108,5 @@ public void setPreferenceLists(String resource,
   protected boolean containsResource(String resource) {
     return _preferenceLists != null && _preferenceLists.containsKey(resource);
   }
+

Review comment:
       remove?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,

Review comment:
       So it should be non-throttled messages, right?
   Could you please find a better name for this list?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/resource/ResourceMessageGenerationPhase.java
##########
@@ -30,8 +32,10 @@
 public class ResourceMessageGenerationPhase extends MessageGenerationPhase {
   @Override
   public void process(ClusterEvent event) throws Exception {
-    IntermediateStateOutput intermediateStateOutput =
-        event.getAttribute(AttributeName.INTERMEDIATE_STATE.name());
-    processEvent(event, intermediateStateOutput);
+    //IntermediateStateOutput intermediateStateOutput =

Review comment:
       remove?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/StateTransitionThrottleController.java
##########
@@ -46,6 +46,20 @@
 
   private boolean _throttleEnabled = false;
 
+  @Override
+  public String toString() {

Review comment:
       Could you please point out where is the usage? I didn't find it.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics

Review comment:
       nit, please keep the original comment, adding TODO above it?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {

Review comment:
       Is this for testing? Make it protected?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       Humm, where is the metrics part? At least we need a TODO here?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,

Review comment:
       Please give it a reasonable name.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;

Review comment:
       This method seems to be added for debug log only.
   For simplifying the business logic, I suggest don't add this method but relying on the existing get method to retrieve information.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_THROTTLED_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THOTTLED_LOAD_MESSAGES

Review comment:
       I guess they are different from the config, @dasahcc 
   But what is the usage of these 2 messages output?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,

Review comment:
       This method is crazily long. It is necessary but is it possible to split "retracedPartitionsStateMap" update out of this method? Basically, I assume it is possible that we update the state map based on the output messages outside, right?
   I think this will help to simplify the code and make it reviewable.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      return selectedResourceMessages;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+    return out;
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(
+      Resource resource,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<Partition, List<Message>> selectedResourceMessages,
+

Review comment:
       format




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541477033



##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -481,6 +482,7 @@ private static PipelineRegistry createDefaultRegistry(String pipelineName) {
       rebalancePipeline.addStage(new MaintenanceRecoveryStage());

Review comment:
       move MaintenanceRecoveryStage after Per replica stages and also changed corresponding input change to MaintenanceRecoveryStage




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482871



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -276,116 +191,173 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
   }
 
   /**
-   * Compute intermediate partition states for a prioritized resource.
-   * @param cache
-   * @param clusterStatusMonitor
-   * @param idealState
-   * @param resource
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
    * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param preferenceLists
-   * @param throttleController
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
    * @return
    */
-  private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDataProvider cache,
-      ClusterStatusMonitor clusterStatusMonitor, IdealState idealState, Resource resource,
-      CurrentStateOutput currentStateOutput, PartitionStateMap bestPossiblePartitionStateMap,
-      Map<String, List<String>> preferenceLists,
-      StateTransitionThrottleController throttleController) {
-    String resourceName = resource.getResourceName();
-    LogUtil.logDebug(logger, _eventId, String.format("Processing resource: %s", resourceName));
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
 
-    // Throttling is applied only on FULL-AUTO mode
-    if (!throttleController.isThrottleEnabled()
-        || !IdealState.RebalanceMode.FULL_AUTO.equals(idealState.getRebalanceMode())) {
-      return bestPossiblePartitionStateMap;
-    }
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    String stateModelDefName = idealState.getStateModelDefRef();
-    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
-    PartitionStateMap intermediatePartitionStateMap = new PartitionStateMap(resourceName);
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
 
-    Set<Partition> partitionsNeedRecovery = new HashSet<>();
-    Set<Partition> partitionsNeedLoadBalance = new HashSet<>();
-    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
-    for (Partition partition : resource.getPartitions()) {
-      Map<String, String> currentStateMap =
-          currentStateOutput.getCurrentStateMap(resourceName, partition);
-      Map<String, String> bestPossibleMap =
-          bestPossiblePartitionStateMap.getPartitionMap(partition);
-      List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    // Resource level prioritization based on the numerical (sortable) priority field.

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r523831097



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,5 +48,8 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  PER_REPLICA_THROTTLED_MESSAGES,
+  PER_REPLICA_RETRACED_STATES

Review comment:
       What does this mean?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555415513



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       Checkout upward seems to be essential. Otherwise, for downward message if it happens `currentCount < expectedCount`, it may classify the message as recovery.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546019612



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       Let me add in the implementation of metrics here too. There is not much change here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] zhangmeng916 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
zhangmeng916 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541462205



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;

Review comment:
       Following our convention, you can write it as _isEmitThrottledMsg = isEmitThrottledMsg;

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -276,116 +191,173 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
   }
 
   /**
-   * Compute intermediate partition states for a prioritized resource.
-   * @param cache
-   * @param clusterStatusMonitor
-   * @param idealState
-   * @param resource
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
    * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param preferenceLists
-   * @param throttleController
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
    * @return
    */
-  private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDataProvider cache,
-      ClusterStatusMonitor clusterStatusMonitor, IdealState idealState, Resource resource,
-      CurrentStateOutput currentStateOutput, PartitionStateMap bestPossiblePartitionStateMap,
-      Map<String, List<String>> preferenceLists,
-      StateTransitionThrottleController throttleController) {
-    String resourceName = resource.getResourceName();
-    LogUtil.logDebug(logger, _eventId, String.format("Processing resource: %s", resourceName));
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
 
-    // Throttling is applied only on FULL-AUTO mode
-    if (!throttleController.isThrottleEnabled()
-        || !IdealState.RebalanceMode.FULL_AUTO.equals(idealState.getRebalanceMode())) {
-      return bestPossiblePartitionStateMap;
-    }
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    String stateModelDefName = idealState.getStateModelDefRef();
-    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
-    PartitionStateMap intermediatePartitionStateMap = new PartitionStateMap(resourceName);
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
 
-    Set<Partition> partitionsNeedRecovery = new HashSet<>();
-    Set<Partition> partitionsNeedLoadBalance = new HashSet<>();
-    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
-    for (Partition partition : resource.getPartitions()) {
-      Map<String, String> currentStateMap =
-          currentStateOutput.getCurrentStateMap(resourceName, partition);
-      Map<String, String> bestPossibleMap =
-          bestPossiblePartitionStateMap.getPartitionMap(partition);
-      List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    // Resource level prioritization based on the numerical (sortable) priority field.

Review comment:
       I cannot tell whether this part has changed or not. But I assume you didn't change any logic for this resource priority calculation?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -396,447 +368,386 @@ private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDa
     // Perform regular load balance only if the number of partitions in recovery and in error is
     // less than the threshold. Otherwise, only allow downward-transition load balance
     boolean onlyDownwardLoadBalance = partitionCount > threshold;
-
-    loadbalanceThrottledPartitions = loadRebalance(resource, currentStateOutput,
-        bestPossiblePartitionStateMap, throttleController, intermediatePartitionStateMap,
-        partitionsNeedLoadBalance, currentStateOutput.getCurrentStateMap(resourceName),
-        onlyDownwardLoadBalance, stateModelDef, cache);
-
-    if (clusterStatusMonitor != null) {
-      clusterStatusMonitor.updateRebalancerStats(resourceName, partitionsNeedRecovery.size(),
-          partitionsNeedLoadBalance.size(), recoveryThrottledPartitions.size(),
-          loadbalanceThrottledPartitions.size());
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
     }
 
-    if (logger.isDebugEnabled()) {
-      logPartitionMapState(resourceName, new HashSet<>(resource.getPartitions()),
-          partitionsNeedRecovery, recoveryThrottledPartitions, partitionsNeedLoadBalance,
-          loadbalanceThrottledPartitions, currentStateOutput, bestPossiblePartitionStateMap,
-          intermediatePartitionStateMap);
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.

Review comment:
       Can you explain a bit more what's in the retraced state map.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -276,116 +191,173 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
   }
 
   /**
-   * Compute intermediate partition states for a prioritized resource.
-   * @param cache
-   * @param clusterStatusMonitor
-   * @param idealState
-   * @param resource
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
    * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param preferenceLists
-   * @param throttleController
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
    * @return
    */
-  private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDataProvider cache,
-      ClusterStatusMonitor clusterStatusMonitor, IdealState idealState, Resource resource,
-      CurrentStateOutput currentStateOutput, PartitionStateMap bestPossiblePartitionStateMap,
-      Map<String, List<String>> preferenceLists,
-      StateTransitionThrottleController throttleController) {
-    String resourceName = resource.getResourceName();
-    LogUtil.logDebug(logger, _eventId, String.format("Processing resource: %s", resourceName));
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
 
-    // Throttling is applied only on FULL-AUTO mode
-    if (!throttleController.isThrottleEnabled()
-        || !IdealState.RebalanceMode.FULL_AUTO.equals(idealState.getRebalanceMode())) {
-      return bestPossiblePartitionStateMap;
-    }
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    String stateModelDefName = idealState.getStateModelDefRef();
-    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
-    PartitionStateMap intermediatePartitionStateMap = new PartitionStateMap(resourceName);
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
 
-    Set<Partition> partitionsNeedRecovery = new HashSet<>();
-    Set<Partition> partitionsNeedLoadBalance = new HashSet<>();
-    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
-    for (Partition partition : resource.getPartitions()) {
-      Map<String, String> currentStateMap =
-          currentStateOutput.getCurrentStateMap(resourceName, partition);
-      Map<String, String> bestPossibleMap =
-          bestPossiblePartitionStateMap.getPartitionMap(partition);
-      List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
 
-      RebalanceType rebalanceType = getRebalanceType(cache, bestPossibleMap, preferenceList,
-          stateModelDef, currentStateMap, idealState, partition.getPartitionName());
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
 
-      // TODO: refine getRebalanceType to return more accurate rebalance types. So the following
-      // logic doesn't need to check for more details.
-      boolean isRebalanceNeeded = false;
+    List<String> failedResources = new ArrayList<>();
 
-      // Check whether partition has any ERROR state replicas
-      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
-        partitionsWithErrorStateReplica.add(partition);
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
       }
 
-      // Number of states required by StateModelDefinition are not satisfied, need recovery
-      if (rebalanceType.equals(RebalanceType.RECOVERY_BALANCE)) {
-        // Check if recovery is needed for this partition
-        if (!currentStateMap.equals(bestPossibleMap)) {
-          partitionsNeedRecovery.add(partition);
-          isRebalanceNeeded = true;
-        }
-      } else if (rebalanceType.equals(RebalanceType.LOAD_BALANCE)) {
-        // Number of states required by StateModelDefinition are satisfied, but to achieve
-        // BestPossibleState, need load balance
-        partitionsNeedLoadBalance.add(partition);
-        isRebalanceNeeded = true;
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
       }
 
-      // Currently at BestPossibleState, no further action necessary
-      if (!isRebalanceNeeded) {
-        Map<String, String> intermediateMap = new HashMap<>(bestPossibleMap);
-        intermediatePartitionStateMap.setState(partition, intermediateMap);
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
       }
     }
 
-    if (!partitionsNeedRecovery.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId, String.format(
-          "Recovery balance needed for %s partitions: %s", resourceName, partitionsNeedRecovery));
-    }
-    if (!partitionsNeedLoadBalance.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId, String.format("Load balance needed for %s partitions: %s",
-          resourceName, partitionsNeedLoadBalance));
-    }
-    if (!partitionsWithErrorStateReplica.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId,
-          String.format("Partition currently has an ERROR replica in %s partitions: %s",
-              resourceName, partitionsWithErrorStateReplica));
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState

Review comment:
       what is this?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -259,14 +181,7 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
                       + " mode due to an instance being assigned more replicas/partitions than "
                       + "the limit.");
             }
-
-            ClusterStatusMonitor clusterStatusMonitor =
-                event.getAttribute(AttributeName.clusterStatusMonitor.name());
-            if (clusterStatusMonitor != null) {
-              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
-                  ResourceMonitor.RebalanceStatus.INTERMEDIATE_STATE_CAL_FAILED);
-            }
-            // Throw an exception here so that messages won't be sent out based on this mapping
+            //TODO: add metrics

Review comment:
       Previous code has `clusterStatusMonitor` already, and I think you can just copy them over. No need to be to TODO right?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",

Review comment:
       "BEST_POSSIBLE_STATE" is stale.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
     }
 
-    IntermediateStateOutput intermediateStateOutput =
-        compute(event, resourceToRebalance, currentStateOutput, bestPossibleStateOutput);
-    event.addAttribute(AttributeName.INTERMEDIATE_STATE.name(), intermediateStateOutput);
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
 
-    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
-    // it does, pause the rebalance and put the cluster on maintenance mode
-    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
-    if (maxPartitionPerInstance > 0) {
-      validateMaxPartitionsPerInstance(event, cache, intermediateStateOutput,
-          maxPartitionPerInstance);
-    }
-  }
-
-  /**
-   * Go through each resource, and based on BestPossibleState and CurrentState, compute
-   * IntermediateState as close to BestPossibleState while maintaining throttling constraints (for
-   * example, ensure that the number of possible pending state transitions does NOT go over the set
-   * threshold).
-   * @param event
-   * @param resourceMap
-   * @param currentStateOutput
-   * @param bestPossibleStateOutput
-   * @return
-   */
-  private IntermediateStateOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput) {
-    IntermediateStateOutput output = new IntermediateStateOutput();
-    ResourceControllerDataProvider dataCache =
-        event.getAttribute(AttributeName.ControllerDataProvider.name());
-
-    StateTransitionThrottleController throttleController = new StateTransitionThrottleController(
-        resourceMap.keySet(), dataCache.getClusterConfig(), dataCache.getLiveInstances().keySet());
-
-    // Resource level prioritization based on the numerical (sortable) priority field.
-    // If the resource priority field is null/not set, the resource will be treated as lowest
-    // priority.
-    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
-    for (String resourceName : resourceMap.keySet()) {
-      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
-    }
-    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
-    // in priority by keeping all priorities at MIN_VALUE
-    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
-      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
-      for (ResourcePriority resourcePriority : prioritizedResourceList) {
-        String resourceName = resourcePriority.getResourceName();
-
-        // Will take the priority from ResourceConfig first
-        // If ResourceConfig does not exist or does not have this field.
-        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
-        if (dataCache.getResourceConfig(resourceName) != null
-            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
-        } else if (dataCache.getIdealState(resourceName) != null && dataCache
-            .getIdealState(resourceName).getRecord().getSimpleField(priorityField) != null) {
-          resourcePriority.setPriority(
-              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
         }
       }
-      prioritizedResourceList.sort(new ResourcePriorityComparator());
     }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
 
-    ClusterStatusMonitor clusterStatusMonitor =
-        event.getAttribute(AttributeName.clusterStatusMonitor.name());
-    List<String> failedResources = new ArrayList<>();
-
-    // Priority is applied in assignment computation because higher priority by looping in order of
-    // decreasing priority
-    for (ResourcePriority resourcePriority : prioritizedResourceList) {
-      String resourceName = resourcePriority.getResourceName();
-
-      if (!bestPossibleStateOutput.containsResource(resourceName)) {
-        LogUtil.logInfo(logger, _eventId, String.format(
-            "Skip calculating intermediate state for resource %s because the best possible state is not available.",
-            resourceName));
-        continue;
-      }
-
-      Resource resource = resourceMap.get(resourceName);
-      IdealState idealState = dataCache.getIdealState(resourceName);
-      if (idealState == null) {
-        // If IdealState is null, use an empty one
-        LogUtil.logInfo(logger, _eventId,
-            String.format(
-                "IdealState for resource %s does not exist; resource may not exist anymore",
-                resourceName));
-        idealState = new IdealState(resourceName);
-        idealState.setStateModelDefRef(resource.getStateModelDefRef());
-      }
-
-      try {
-        output.setState(resourceName,
-            computeIntermediatePartitionState(dataCache, clusterStatusMonitor, idealState,
-                resourceMap.get(resourceName), currentStateOutput,
-                bestPossibleStateOutput.getPartitionStateMap(resourceName),
-                bestPossibleStateOutput.getPreferenceLists(resourceName), throttleController));
-      } catch (HelixException ex) {
-        LogUtil.logInfo(logger, _eventId,
-            "Failed to calculate intermediate partition states for resource " + resourceName, ex);
-        failedResources.add(resourceName);
-      }
+    if (isEmitThrottledMsg) {

Review comment:
       This does not sounds reasonable to me. Having this code in formal release is weird. And how do you plan to enable it? You still need a code change to check in right?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -396,447 +368,386 @@ private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDa
     // Perform regular load balance only if the number of partitions in recovery and in error is
     // less than the threshold. Otherwise, only allow downward-transition load balance
     boolean onlyDownwardLoadBalance = partitionCount > threshold;
-
-    loadbalanceThrottledPartitions = loadRebalance(resource, currentStateOutput,
-        bestPossiblePartitionStateMap, throttleController, intermediatePartitionStateMap,
-        partitionsNeedLoadBalance, currentStateOutput.getCurrentStateMap(resourceName),
-        onlyDownwardLoadBalance, stateModelDef, cache);
-
-    if (clusterStatusMonitor != null) {
-      clusterStatusMonitor.updateRebalancerStats(resourceName, partitionsNeedRecovery.size(),
-          partitionsNeedLoadBalance.size(), recoveryThrottledPartitions.size(),
-          loadbalanceThrottledPartitions.size());
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
     }
 
-    if (logger.isDebugEnabled()) {
-      logPartitionMapState(resourceName, new HashSet<>(resource.getPartitions()),
-          partitionsNeedRecovery, recoveryThrottledPartitions, partitionsNeedLoadBalance,
-          loadbalanceThrottledPartitions, currentStateOutput, bestPossiblePartitionStateMap,
-          intermediatePartitionStateMap);
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
     }
-
-    LogUtil.logDebug(logger, _eventId, String.format("End processing resource: %s", resourceName));
-    return intermediatePartitionStateMap;
+    return out;
   }
 
-  /**
-   * Check for a partition, whether all transitions for its replicas are downward transitions. Note
-   * that this function does NOT check for ERROR states.
-   * @param currentStateMap
-   * @param bestPossibleMap
-   * @param stateModelDef
-   * @return true if there are; false otherwise
-   */
-  private boolean isLoadBalanceDownwardForAllReplicas(Map<String, String> currentStateMap,
-      Map<String, String> bestPossibleMap, StateModelDefinition stateModelDef) {
-    Set<String> allInstances = new HashSet<>();
-    allInstances.addAll(currentStateMap.keySet());
-    allInstances.addAll(bestPossibleMap.keySet());
-    Map<String, Integer> statePriorityMap = stateModelDef.getStatePriorityMap();
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+  ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
 
-    for (String instance : allInstances) {
-      String currentState = currentStateMap.get(instance);
-      String bestPossibleState = bestPossibleMap.get(instance);
-      if (currentState == null) {
-        return false; // null -> state is upward
-      }
-      if (bestPossibleState != null) {
-        // Compare priority values and return if an upward transition is found
-        // Note that lower integer value implies higher priority
-        if (!statePriorityMap.containsKey(currentState)
-            || !statePriorityMap.containsKey(bestPossibleState)) {
-          // If the state is not found in statePriorityMap, consider it not strictly downward by
-          // default because we can't determine whether it is downward
-          return false;
-        }
-        if (statePriorityMap.get(currentState) > statePriorityMap.get(bestPossibleState)) {
-          return false;
-        }
-      }
-    }
-    return true;
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
   }
 
-  /**
-   * Check and charge all pending transitions for throttling.
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
    */
-  private void chargePendingTransition(Resource resource, CurrentStateOutput currentStateOutput,
-      StateTransitionThrottleController throttleController, Set<Partition> partitionsNeedRecovery,
-      Set<Partition> partitionsNeedLoadbalance, ResourceControllerDataProvider cache,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      PartitionStateMap intermediatePartitionStateMap) {
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
     String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
 
-    // check and charge pending transitions
     for (Partition partition : resource.getPartitions()) {
-      // Maps instance to its current state
       Map<String, String> currentStateMap =
           currentStateOutput.getCurrentStateMap(resourceName, partition);
-      // Maps instance to its pending (next) state
-      Map<String, String> pendingMap =
-          currentStateOutput.getPendingStateMap(resourceName, partition);
-
-      StateTransitionThrottleConfig.RebalanceType rebalanceType = RebalanceType.NONE;
-      if (partitionsNeedRecovery.contains(partition)) {
-        rebalanceType = StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
-      } else if (partitionsNeedLoadbalance.contains(partition)) {
-        rebalanceType = StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE;
-      }
-
-      if (pendingMap.size() > 0) {
-        boolean shouldChargePartition = false;
-        for (String instance : pendingMap.keySet()) {
-          String currentState = currentStateMap.get(instance);
-          String pendingState = pendingMap.get(instance);
-          if (pendingState != null && !pendingState.equals(currentState)
-              && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-              .contains(instance)) {
-            // Only charge this instance if the partition is not disabled
-            throttleController.chargeInstance(rebalanceType, instance);
-            shouldChargePartition = true;
-            // If there is a pending state transition for the partition, that means that an assignment
-            // has already been made and the state transition message has already been sent out for the partition
-            // in a previous pipeline run. We must honor this and reflect it by charging for the pending state transition message.
-
-            // Since the assignment has already been made for the pending message, we do a special treatment
-            // for it by setting the best possible state directly in intermediatePartitionStateMap so that the pending
-            // message won't be double-assigned or double-charged in recovery or load balance.
-            handlePendingStateTransitionsForThrottling(partition, partitionsNeedRecovery,
-                partitionsNeedLoadbalance, rebalanceType, bestPossiblePartitionStateMap,
-                intermediatePartitionStateMap);
-          }
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
         }
-        if (shouldChargePartition) {
-          throttleController.chargeCluster(rebalanceType);
-          throttleController.chargeResource(rebalanceType, resourceName);
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
         }
       }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
     }
-  }
-
-  /**
-   * Sort partitions according to partition priority {@link PartitionPriorityComparator}, and for
-   * each partition, throttle state transitions if needed. Also populate
-   * intermediatePartitionStateMap either with BestPossibleState (if no throttling is necessary) or
-   * CurrentState (if throttled).
-   * @param resource
-   * @param bestPossiblePartitionStateMap
-   * @param throttleController
-   * @param intermediatePartitionStateMap
-   * @param partitionsNeedRecovery
-   * @param currentStateOutput
-   * @param topState
-   * @param cache
-   * @return a set of partitions that need recovery but did not get recovered due to throttling
-   */
-  private Set<Partition> recoveryRebalance(Resource resource,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      StateTransitionThrottleController throttleController,
-      PartitionStateMap intermediatePartitionStateMap, Set<Partition> partitionsNeedRecovery,
-      CurrentStateOutput currentStateOutput, String topState,
-      ResourceControllerDataProvider cache) {
-    String resourceName = resource.getResourceName();
-    Set<Partition> partitionRecoveryBalanceThrottled = new HashSet<>();
 
-    // Maps Partition -> Instance -> State
-    Map<Partition, Map<String, String>> currentStateMap =
-        currentStateOutput.getCurrentStateMap(resourceName);
-    List<Partition> partitionsNeedRecoveryPrioritized = new ArrayList<>(partitionsNeedRecovery);
-
-    // We want the result of the intermediate state calculation to be deterministic. We sort here by
-    // partition name to ensure that the order is consistent for inputs fed into
-    // PartitionPriorityComparator sort
-    partitionsNeedRecoveryPrioritized.sort(Comparator.comparing(Partition::getPartitionName));
-    partitionsNeedRecoveryPrioritized.sort(new PartitionPriorityComparator(
-        bestPossiblePartitionStateMap.getStateMap(), currentStateMap, topState, true));
-
-    // For each partition, apply throttling if needed.
-    for (Partition partition : partitionsNeedRecoveryPrioritized) {
-      throttleStateTransitionsForPartition(throttleController, resourceName, partition,
-          currentStateOutput, bestPossiblePartitionStateMap, partitionRecoveryBalanceThrottled,
-          intermediatePartitionStateMap, RebalanceType.RECOVERY_BALANCE, cache);
-    }
-    LogUtil.logInfo(logger, _eventId, String.format(
-        "For resource %s: Num of partitions needing recovery: %d, Num of partitions needing recovery"
-            + " but throttled (not recovered): %d",
-        resourceName, partitionsNeedRecovery.size(), partitionRecoveryBalanceThrottled.size()));
-    return partitionRecoveryBalanceThrottled;
   }
 
-  /**
-   * Sort partitions according to partition priority {@link PartitionPriorityComparator}, and for
-   * each partition, throttle state transitions if needed. Also populate
-   * intermediatePartitionStateMap either with BestPossibleState (if no throttling is necessary) or
-   * CurrentState (if throttled).
-   * @param resource
-   * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param throttleController
-   * @param intermediatePartitionStateMap
-   * @param partitionsNeedLoadbalance
-   * @param currentStateMap
-   * @param onlyDownwardLoadBalance true when only allowing downward transitions
-   * @param stateModelDef for determining whether a partition's transitions are strictly downward
-   * @param cache
-   * @return
+  /*
+   * Classify the messages of each partition into recovery and load messages.
    */
-  private Set<Partition> loadRebalance(Resource resource, CurrentStateOutput currentStateOutput,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      StateTransitionThrottleController throttleController,
-      PartitionStateMap intermediatePartitionStateMap, Set<Partition> partitionsNeedLoadbalance,
-      Map<Partition, Map<String, String>> currentStateMap, boolean onlyDownwardLoadBalance,
-      StateModelDefinition stateModelDef, ResourceControllerDataProvider cache) {
+  private void classifyMessages(
+      Resource resource,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<Partition, List<Message>> selectedResourceMessages,
+
+      List<Message> recoveryMessages,
+      List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap
+  ) {
+
     String resourceName = resource.getResourceName();
-    Set<Partition> partitionsLoadbalanceThrottled = new HashSet<>();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
 
-    List<Partition> partitionsNeedLoadRebalancePrioritized =
-        new ArrayList<>(partitionsNeedLoadbalance);
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
 
-    // We want the result of the intermediate state calculation to be deterministic. We sort here by
-    // partition name to ensure that the order is consistent for inputs fed into
-    // PartitionPriorityComparator sort
-    partitionsNeedLoadRebalancePrioritized.sort(Comparator.comparing(Partition::getPartitionName));
-    partitionsNeedLoadRebalancePrioritized.sort(new PartitionPriorityComparator(
-        bestPossiblePartitionStateMap.getStateMap(), currentStateMap, "", false));
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
 
-    for (Partition partition : partitionsNeedLoadRebalancePrioritized) {
-      // If this is a downward load balance, check if the partition's transition is strictly
-      // downward
-      if (onlyDownwardLoadBalance) {
-        Map<String, String> currentStateMapForPartition =
-            currentStateOutput.getCurrentStateMap(resourceName, partition);
-        Map<String, String> bestPossibleMapForPartition =
-            bestPossiblePartitionStateMap.getPartitionMap(partition);
-        if (!isLoadBalanceDownwardForAllReplicas(currentStateMapForPartition,
-            bestPossibleMapForPartition, stateModelDef)) {
-          // For downward load balance, if a partition's transitions are not strictly downward,
-          // set currentState to intermediateState
-          intermediatePartitionStateMap.setState(partition, currentStateMapForPartition);
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
           continue;
         }
-      }
-      throttleStateTransitionsForPartition(throttleController, resourceName, partition,
-          currentStateOutput, bestPossiblePartitionStateMap, partitionsLoadbalanceThrottled,
-          intermediatePartitionStateMap, RebalanceType.LOAD_BALANCE, cache);
-    }
-    LogUtil.logInfo(logger, _eventId,
-        String.format(
-            "For resource %s: Num of partitions needing load-balance: %d, Num of partitions needing"
-                + " load-balance but throttled (not load-balanced): %d",
-            resourceName, partitionsNeedLoadbalance.size(), partitionsLoadbalanceThrottled.size()));
-    return partitionsLoadbalanceThrottled;
-  }
 
-  /**
-   * Check the status on throttling at every level (cluster, resource, instance) and set
-   * intermediatePartitionStateMap accordingly per partition.
-   * @param throttleController
-   * @param resourceName
-   * @param partition
-   * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param partitionsThrottled
-   * @param intermediatePartitionStateMap
-   * @param rebalanceType
-   * @param cache
-   */
-  private void throttleStateTransitionsForPartition(
-      StateTransitionThrottleController throttleController, String resourceName,
-      Partition partition, CurrentStateOutput currentStateOutput,
-      PartitionStateMap bestPossiblePartitionStateMap, Set<Partition> partitionsThrottled,
-      PartitionStateMap intermediatePartitionStateMap, RebalanceType rebalanceType,
-      ResourceControllerDataProvider cache) {
-
-    Map<String, String> currentStateMap =
-        currentStateOutput.getCurrentStateMap(resourceName, partition);
-    Map<String, String> bestPossibleMap = bestPossiblePartitionStateMap.getPartitionMap(partition);
-    Set<String> allInstances = new HashSet<>(currentStateMap.keySet());
-    allInstances.addAll(bestPossibleMap.keySet());
-    Map<String, String> intermediateMap = new HashMap<>();
-
-    boolean hasReachedThrottlingLimit = false;
-    if (throttleController.shouldThrottleForResource(rebalanceType, resourceName)) {
-      hasReachedThrottlingLimit = true;
-      if (logger.isDebugEnabled()) {
-        LogUtil.logDebug(logger, _eventId,
-            String.format("Throttled on partition: %s in resource: %s",
-                partition.getPartitionName(), resourceName));
-      }
-    } else {
-      // throttle if any of the instances are not able to accept state transitions
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          if (throttleController.shouldThrottleForInstance(rebalanceType, instance)) {
-            hasReachedThrottlingLimit = true;
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
             if (logger.isDebugEnabled()) {
               LogUtil.logDebug(logger, _eventId,
-                  String.format(
-                      "Throttled because of instance: %s for partition: %s in resource: %s",
-                      instance, partition.getPartitionName(), resourceName));
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
             }
-            break;
+            continue;
           }
         }
-      }
-    }
-    if (!hasReachedThrottlingLimit) {
-      // This implies that there is room for more state transitions.
-      // Find instances with a replica whose current state is different from BestPossibleState and
-      // "charge" for it, and bestPossibleStates will become intermediate states
-      intermediateMap.putAll(bestPossibleMap);
-      boolean shouldChargeForPartition = false;
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          throttleController.chargeInstance(rebalanceType, instance);
-          shouldChargeForPartition = true;
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
         }
-      }
-      if (shouldChargeForPartition) {
-        throttleController.chargeCluster(rebalanceType);
-        throttleController.chargeResource(rebalanceType, resourceName);
-      }
-    } else {
-      // No more room for more state transitions; current states will just become intermediate
-      // states unless the partition is disabled
-      // Add this partition to a set of throttled partitions
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          // Because this partition is disabled, we allow assignment
-          intermediateMap.put(instance, bestPossibleState);
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
         } else {
-          // This partition is not disabled, so it must be throttled by just passing on the current
-          // state
-          if (currentState != null) {
-            intermediateMap.put(instance, currentState);
-          }
-          partitionsThrottled.add(partition);
+          loadMessages.add(msg);
         }
       }
     }
-    intermediatePartitionStateMap.setState(partition, intermediateMap);
   }
 
-  /**
-   * For a partition, given its preferenceList, bestPossibleState, and currentState, determine which
-   * type of rebalance is needed to model IdealState's states defined by the state model definition.
-   * @return RebalanceType needed to bring the replicas to idea states
-   *         RECOVERY_BALANCE - not all required states (replicas) are available through all
-   *         replicas, or the partition is disabled
-   *         NONE - current state matches the ideal state
-   *         LOAD_BALANCE - although all replicas required exist, Helix needs to optimize the
-   *         allocation
-   */
-  private RebalanceType getRebalanceType(ResourceControllerDataProvider cache,
-      Map<String, String> bestPossibleMap, List<String> preferenceList,
-      StateModelDefinition stateModelDef, Map<String, String> currentStateMap,
-      IdealState idealState, String partitionName) {
-    if (preferenceList == null) {
-      preferenceList = Collections.emptyList();
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+  ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {

Review comment:
       This check seems redundant. This function is only used internally, and when you call it, you know it won't happen.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r545320609



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       This could potentially have problem. The sorting algorithm should be same as last round.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       We should think about the general case instead of only single top state and secondary state case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541377436



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,

Review comment:
       Sure. I will add another private method for retraced map handling. Basically step 6 would be in a separate method.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] narendly commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
narendly commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r570636751



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/AttributeName.java
##########
@@ -48,7 +48,20 @@
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
   TO_BE_PURGED_WORKFLOWS,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
+
   JOBS_WITHOUT_CONFIG,
   // This attribute should only be used in TaskGarbageCollectionStage, misuse could cause race conditions.
-  TO_BE_PURGED_JOBS_MAP
+  TO_BE_PURGED_JOBS_MAP,
+
+  // This attribute denotes the messages output from Per Preplica Throttle stage
+  PER_REPLICA_OUTPUT_MESSAGES,
+
+  // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
+  PER_REPLICA_RETRACED_STATES,
+
+  // This attribute denotes the filtered out messages deemed as recovery message
+  PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
+
+  // This attribute denotes the filtered out messages deemed as load message
+  PER_REPLICA_THROTTLED_LOAD_MESSAGES

Review comment:
       ```suggestion
     TO_BE_PURGED_JOBS_MAP,
   
     // This attribute denotes the messages output from Per Preplica Throttle stage
     PER_REPLICA_OUTPUT_MESSAGES,
   
     // This attribute denotes the targeted partition state mapping from Per Preplica Throttle stage
     PER_REPLICA_RETRACED_STATES,
   
     // This attribute denotes the filtered out messages deemed as recovery message
     PER_REPLICA_THROTTLED_RECOVERY_MESSAGES,
   
     // This attribute denotes the filtered out messages deemed as load message
     PER_REPLICA_THROTTLED_LOAD_MESSAGES
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r562228681



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());

Review comment:
       Tested, this is a good point, we should have some check as original code. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r554258183



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       accumulation logic is added in `getPartitionExpectedAndCurrentStateCountMap`. thus the testing logic is simplified a lot here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       This is a good suggestion to make the code more generic. The accumulation logic is added in `getPartitionExpectedAndCurrentStateCountMap` and the testing of load/recovery logic is simplified a lot here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));

Review comment:
       Basically `PartitionMessageComparator` is only used to sort message (replica) with a partition. This logic is not need in previous IntermediateCalStage. 
   `MessageThrottleComparator` is use to determine the message order across partitions and this is the same logic as PartitionPriorityComparator in IntermediateCalcStage.
   
   What is you take?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541406530



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482967



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -276,116 +191,173 @@ private void validateMaxPartitionsPerInstance(ClusterEvent event,
   }
 
   /**
-   * Compute intermediate partition states for a prioritized resource.
-   * @param cache
-   * @param clusterStatusMonitor
-   * @param idealState
-   * @param resource
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
    * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param preferenceLists
-   * @param throttleController
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
    * @return
    */
-  private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDataProvider cache,
-      ClusterStatusMonitor clusterStatusMonitor, IdealState idealState, Resource resource,
-      CurrentStateOutput currentStateOutput, PartitionStateMap bestPossiblePartitionStateMap,
-      Map<String, List<String>> preferenceLists,
-      StateTransitionThrottleController throttleController) {
-    String resourceName = resource.getResourceName();
-    LogUtil.logDebug(logger, _eventId, String.format("Processing resource: %s", resourceName));
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
 
-    // Throttling is applied only on FULL-AUTO mode
-    if (!throttleController.isThrottleEnabled()
-        || !IdealState.RebalanceMode.FULL_AUTO.equals(idealState.getRebalanceMode())) {
-      return bestPossiblePartitionStateMap;
-    }
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    String stateModelDefName = idealState.getStateModelDefRef();
-    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
-    PartitionStateMap intermediatePartitionStateMap = new PartitionStateMap(resourceName);
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
 
-    Set<Partition> partitionsNeedRecovery = new HashSet<>();
-    Set<Partition> partitionsNeedLoadBalance = new HashSet<>();
-    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
-    for (Partition partition : resource.getPartitions()) {
-      Map<String, String> currentStateMap =
-          currentStateOutput.getCurrentStateMap(resourceName, partition);
-      Map<String, String> bestPossibleMap =
-          bestPossiblePartitionStateMap.getPartitionMap(partition);
-      List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
 
-      RebalanceType rebalanceType = getRebalanceType(cache, bestPossibleMap, preferenceList,
-          stateModelDef, currentStateMap, idealState, partition.getPartitionName());
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
 
-      // TODO: refine getRebalanceType to return more accurate rebalance types. So the following
-      // logic doesn't need to check for more details.
-      boolean isRebalanceNeeded = false;
+    List<String> failedResources = new ArrayList<>();
 
-      // Check whether partition has any ERROR state replicas
-      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
-        partitionsWithErrorStateReplica.add(partition);
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
       }
 
-      // Number of states required by StateModelDefinition are not satisfied, need recovery
-      if (rebalanceType.equals(RebalanceType.RECOVERY_BALANCE)) {
-        // Check if recovery is needed for this partition
-        if (!currentStateMap.equals(bestPossibleMap)) {
-          partitionsNeedRecovery.add(partition);
-          isRebalanceNeeded = true;
-        }
-      } else if (rebalanceType.equals(RebalanceType.LOAD_BALANCE)) {
-        // Number of states required by StateModelDefinition are satisfied, but to achieve
-        // BestPossibleState, need load balance
-        partitionsNeedLoadBalance.add(partition);
-        isRebalanceNeeded = true;
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
       }
 
-      // Currently at BestPossibleState, no further action necessary
-      if (!isRebalanceNeeded) {
-        Map<String, String> intermediateMap = new HashMap<>(bestPossibleMap);
-        intermediatePartitionStateMap.setState(partition, intermediateMap);
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
       }
     }
 
-    if (!partitionsNeedRecovery.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId, String.format(
-          "Recovery balance needed for %s partitions: %s", resourceName, partitionsNeedRecovery));
-    }
-    if (!partitionsNeedLoadBalance.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId, String.format("Load balance needed for %s partitions: %s",
-          resourceName, partitionsNeedLoadBalance));
-    }
-    if (!partitionsWithErrorStateReplica.isEmpty()) {
-      LogUtil.logInfo(logger, _eventId,
-          String.format("Partition currently has an ERROR replica in %s partitions: %s",
-              resourceName, partitionsWithErrorStateReplica));
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -396,447 +368,386 @@ private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDa
     // Perform regular load balance only if the number of partitions in recovery and in error is
     // less than the threshold. Otherwise, only allow downward-transition load balance
     boolean onlyDownwardLoadBalance = partitionCount > threshold;
-
-    loadbalanceThrottledPartitions = loadRebalance(resource, currentStateOutput,
-        bestPossiblePartitionStateMap, throttleController, intermediatePartitionStateMap,
-        partitionsNeedLoadBalance, currentStateOutput.getCurrentStateMap(resourceName),
-        onlyDownwardLoadBalance, stateModelDef, cache);
-
-    if (clusterStatusMonitor != null) {
-      clusterStatusMonitor.updateRebalancerStats(resourceName, partitionsNeedRecovery.size(),
-          partitionsNeedLoadBalance.size(), recoveryThrottledPartitions.size(),
-          loadbalanceThrottledPartitions.size());
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
     }
 
-    if (logger.isDebugEnabled()) {
-      logPartitionMapState(resourceName, new HashSet<>(resource.getPartitions()),
-          partitionsNeedRecovery, recoveryThrottledPartitions, partitionsNeedLoadBalance,
-          loadbalanceThrottledPartitions, currentStateOutput, bestPossiblePartitionStateMap,
-          intermediatePartitionStateMap);
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
     }
-
-    LogUtil.logDebug(logger, _eventId, String.format("End processing resource: %s", resourceName));
-    return intermediatePartitionStateMap;
+    return out;
   }
 
-  /**
-   * Check for a partition, whether all transitions for its replicas are downward transitions. Note
-   * that this function does NOT check for ERROR states.
-   * @param currentStateMap
-   * @param bestPossibleMap
-   * @param stateModelDef
-   * @return true if there are; false otherwise
-   */
-  private boolean isLoadBalanceDownwardForAllReplicas(Map<String, String> currentStateMap,
-      Map<String, String> bestPossibleMap, StateModelDefinition stateModelDef) {
-    Set<String> allInstances = new HashSet<>();
-    allInstances.addAll(currentStateMap.keySet());
-    allInstances.addAll(bestPossibleMap.keySet());
-    Map<String, Integer> statePriorityMap = stateModelDef.getStatePriorityMap();
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+  ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
 
-    for (String instance : allInstances) {
-      String currentState = currentStateMap.get(instance);
-      String bestPossibleState = bestPossibleMap.get(instance);
-      if (currentState == null) {
-        return false; // null -> state is upward
-      }
-      if (bestPossibleState != null) {
-        // Compare priority values and return if an upward transition is found
-        // Note that lower integer value implies higher priority
-        if (!statePriorityMap.containsKey(currentState)
-            || !statePriorityMap.containsKey(bestPossibleState)) {
-          // If the state is not found in statePriorityMap, consider it not strictly downward by
-          // default because we can't determine whether it is downward
-          return false;
-        }
-        if (statePriorityMap.get(currentState) > statePriorityMap.get(bestPossibleState)) {
-          return false;
-        }
-      }
-    }
-    return true;
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
   }
 
-  /**
-   * Check and charge all pending transitions for throttling.
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
    */
-  private void chargePendingTransition(Resource resource, CurrentStateOutput currentStateOutput,
-      StateTransitionThrottleController throttleController, Set<Partition> partitionsNeedRecovery,
-      Set<Partition> partitionsNeedLoadbalance, ResourceControllerDataProvider cache,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      PartitionStateMap intermediatePartitionStateMap) {
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
     String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
 
-    // check and charge pending transitions
     for (Partition partition : resource.getPartitions()) {
-      // Maps instance to its current state
       Map<String, String> currentStateMap =
           currentStateOutput.getCurrentStateMap(resourceName, partition);
-      // Maps instance to its pending (next) state
-      Map<String, String> pendingMap =
-          currentStateOutput.getPendingStateMap(resourceName, partition);
-
-      StateTransitionThrottleConfig.RebalanceType rebalanceType = RebalanceType.NONE;
-      if (partitionsNeedRecovery.contains(partition)) {
-        rebalanceType = StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
-      } else if (partitionsNeedLoadbalance.contains(partition)) {
-        rebalanceType = StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE;
-      }
-
-      if (pendingMap.size() > 0) {
-        boolean shouldChargePartition = false;
-        for (String instance : pendingMap.keySet()) {
-          String currentState = currentStateMap.get(instance);
-          String pendingState = pendingMap.get(instance);
-          if (pendingState != null && !pendingState.equals(currentState)
-              && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-              .contains(instance)) {
-            // Only charge this instance if the partition is not disabled
-            throttleController.chargeInstance(rebalanceType, instance);
-            shouldChargePartition = true;
-            // If there is a pending state transition for the partition, that means that an assignment
-            // has already been made and the state transition message has already been sent out for the partition
-            // in a previous pipeline run. We must honor this and reflect it by charging for the pending state transition message.
-
-            // Since the assignment has already been made for the pending message, we do a special treatment
-            // for it by setting the best possible state directly in intermediatePartitionStateMap so that the pending
-            // message won't be double-assigned or double-charged in recovery or load balance.
-            handlePendingStateTransitionsForThrottling(partition, partitionsNeedRecovery,
-                partitionsNeedLoadbalance, rebalanceType, bestPossiblePartitionStateMap,
-                intermediatePartitionStateMap);
-          }
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
         }
-        if (shouldChargePartition) {
-          throttleController.chargeCluster(rebalanceType);
-          throttleController.chargeResource(rebalanceType, resourceName);
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
         }
       }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
     }
-  }
-
-  /**
-   * Sort partitions according to partition priority {@link PartitionPriorityComparator}, and for
-   * each partition, throttle state transitions if needed. Also populate
-   * intermediatePartitionStateMap either with BestPossibleState (if no throttling is necessary) or
-   * CurrentState (if throttled).
-   * @param resource
-   * @param bestPossiblePartitionStateMap
-   * @param throttleController
-   * @param intermediatePartitionStateMap
-   * @param partitionsNeedRecovery
-   * @param currentStateOutput
-   * @param topState
-   * @param cache
-   * @return a set of partitions that need recovery but did not get recovered due to throttling
-   */
-  private Set<Partition> recoveryRebalance(Resource resource,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      StateTransitionThrottleController throttleController,
-      PartitionStateMap intermediatePartitionStateMap, Set<Partition> partitionsNeedRecovery,
-      CurrentStateOutput currentStateOutput, String topState,
-      ResourceControllerDataProvider cache) {
-    String resourceName = resource.getResourceName();
-    Set<Partition> partitionRecoveryBalanceThrottled = new HashSet<>();
 
-    // Maps Partition -> Instance -> State
-    Map<Partition, Map<String, String>> currentStateMap =
-        currentStateOutput.getCurrentStateMap(resourceName);
-    List<Partition> partitionsNeedRecoveryPrioritized = new ArrayList<>(partitionsNeedRecovery);
-
-    // We want the result of the intermediate state calculation to be deterministic. We sort here by
-    // partition name to ensure that the order is consistent for inputs fed into
-    // PartitionPriorityComparator sort
-    partitionsNeedRecoveryPrioritized.sort(Comparator.comparing(Partition::getPartitionName));
-    partitionsNeedRecoveryPrioritized.sort(new PartitionPriorityComparator(
-        bestPossiblePartitionStateMap.getStateMap(), currentStateMap, topState, true));
-
-    // For each partition, apply throttling if needed.
-    for (Partition partition : partitionsNeedRecoveryPrioritized) {
-      throttleStateTransitionsForPartition(throttleController, resourceName, partition,
-          currentStateOutput, bestPossiblePartitionStateMap, partitionRecoveryBalanceThrottled,
-          intermediatePartitionStateMap, RebalanceType.RECOVERY_BALANCE, cache);
-    }
-    LogUtil.logInfo(logger, _eventId, String.format(
-        "For resource %s: Num of partitions needing recovery: %d, Num of partitions needing recovery"
-            + " but throttled (not recovered): %d",
-        resourceName, partitionsNeedRecovery.size(), partitionRecoveryBalanceThrottled.size()));
-    return partitionRecoveryBalanceThrottled;
   }
 
-  /**
-   * Sort partitions according to partition priority {@link PartitionPriorityComparator}, and for
-   * each partition, throttle state transitions if needed. Also populate
-   * intermediatePartitionStateMap either with BestPossibleState (if no throttling is necessary) or
-   * CurrentState (if throttled).
-   * @param resource
-   * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param throttleController
-   * @param intermediatePartitionStateMap
-   * @param partitionsNeedLoadbalance
-   * @param currentStateMap
-   * @param onlyDownwardLoadBalance true when only allowing downward transitions
-   * @param stateModelDef for determining whether a partition's transitions are strictly downward
-   * @param cache
-   * @return
+  /*
+   * Classify the messages of each partition into recovery and load messages.
    */
-  private Set<Partition> loadRebalance(Resource resource, CurrentStateOutput currentStateOutput,
-      PartitionStateMap bestPossiblePartitionStateMap,
-      StateTransitionThrottleController throttleController,
-      PartitionStateMap intermediatePartitionStateMap, Set<Partition> partitionsNeedLoadbalance,
-      Map<Partition, Map<String, String>> currentStateMap, boolean onlyDownwardLoadBalance,
-      StateModelDefinition stateModelDef, ResourceControllerDataProvider cache) {
+  private void classifyMessages(
+      Resource resource,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<Partition, List<Message>> selectedResourceMessages,
+
+      List<Message> recoveryMessages,
+      List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap
+  ) {
+
     String resourceName = resource.getResourceName();
-    Set<Partition> partitionsLoadbalanceThrottled = new HashSet<>();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
 
-    List<Partition> partitionsNeedLoadRebalancePrioritized =
-        new ArrayList<>(partitionsNeedLoadbalance);
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
 
-    // We want the result of the intermediate state calculation to be deterministic. We sort here by
-    // partition name to ensure that the order is consistent for inputs fed into
-    // PartitionPriorityComparator sort
-    partitionsNeedLoadRebalancePrioritized.sort(Comparator.comparing(Partition::getPartitionName));
-    partitionsNeedLoadRebalancePrioritized.sort(new PartitionPriorityComparator(
-        bestPossiblePartitionStateMap.getStateMap(), currentStateMap, "", false));
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
 
-    for (Partition partition : partitionsNeedLoadRebalancePrioritized) {
-      // If this is a downward load balance, check if the partition's transition is strictly
-      // downward
-      if (onlyDownwardLoadBalance) {
-        Map<String, String> currentStateMapForPartition =
-            currentStateOutput.getCurrentStateMap(resourceName, partition);
-        Map<String, String> bestPossibleMapForPartition =
-            bestPossiblePartitionStateMap.getPartitionMap(partition);
-        if (!isLoadBalanceDownwardForAllReplicas(currentStateMapForPartition,
-            bestPossibleMapForPartition, stateModelDef)) {
-          // For downward load balance, if a partition's transitions are not strictly downward,
-          // set currentState to intermediateState
-          intermediatePartitionStateMap.setState(partition, currentStateMapForPartition);
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
           continue;
         }
-      }
-      throttleStateTransitionsForPartition(throttleController, resourceName, partition,
-          currentStateOutput, bestPossiblePartitionStateMap, partitionsLoadbalanceThrottled,
-          intermediatePartitionStateMap, RebalanceType.LOAD_BALANCE, cache);
-    }
-    LogUtil.logInfo(logger, _eventId,
-        String.format(
-            "For resource %s: Num of partitions needing load-balance: %d, Num of partitions needing"
-                + " load-balance but throttled (not load-balanced): %d",
-            resourceName, partitionsNeedLoadbalance.size(), partitionsLoadbalanceThrottled.size()));
-    return partitionsLoadbalanceThrottled;
-  }
 
-  /**
-   * Check the status on throttling at every level (cluster, resource, instance) and set
-   * intermediatePartitionStateMap accordingly per partition.
-   * @param throttleController
-   * @param resourceName
-   * @param partition
-   * @param currentStateOutput
-   * @param bestPossiblePartitionStateMap
-   * @param partitionsThrottled
-   * @param intermediatePartitionStateMap
-   * @param rebalanceType
-   * @param cache
-   */
-  private void throttleStateTransitionsForPartition(
-      StateTransitionThrottleController throttleController, String resourceName,
-      Partition partition, CurrentStateOutput currentStateOutput,
-      PartitionStateMap bestPossiblePartitionStateMap, Set<Partition> partitionsThrottled,
-      PartitionStateMap intermediatePartitionStateMap, RebalanceType rebalanceType,
-      ResourceControllerDataProvider cache) {
-
-    Map<String, String> currentStateMap =
-        currentStateOutput.getCurrentStateMap(resourceName, partition);
-    Map<String, String> bestPossibleMap = bestPossiblePartitionStateMap.getPartitionMap(partition);
-    Set<String> allInstances = new HashSet<>(currentStateMap.keySet());
-    allInstances.addAll(bestPossibleMap.keySet());
-    Map<String, String> intermediateMap = new HashMap<>();
-
-    boolean hasReachedThrottlingLimit = false;
-    if (throttleController.shouldThrottleForResource(rebalanceType, resourceName)) {
-      hasReachedThrottlingLimit = true;
-      if (logger.isDebugEnabled()) {
-        LogUtil.logDebug(logger, _eventId,
-            String.format("Throttled on partition: %s in resource: %s",
-                partition.getPartitionName(), resourceName));
-      }
-    } else {
-      // throttle if any of the instances are not able to accept state transitions
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          if (throttleController.shouldThrottleForInstance(rebalanceType, instance)) {
-            hasReachedThrottlingLimit = true;
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
             if (logger.isDebugEnabled()) {
               LogUtil.logDebug(logger, _eventId,
-                  String.format(
-                      "Throttled because of instance: %s for partition: %s in resource: %s",
-                      instance, partition.getPartitionName(), resourceName));
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
             }
-            break;
+            continue;
           }
         }
-      }
-    }
-    if (!hasReachedThrottlingLimit) {
-      // This implies that there is room for more state transitions.
-      // Find instances with a replica whose current state is different from BestPossibleState and
-      // "charge" for it, and bestPossibleStates will become intermediate states
-      intermediateMap.putAll(bestPossibleMap);
-      boolean shouldChargeForPartition = false;
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && !cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          throttleController.chargeInstance(rebalanceType, instance);
-          shouldChargeForPartition = true;
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
         }
-      }
-      if (shouldChargeForPartition) {
-        throttleController.chargeCluster(rebalanceType);
-        throttleController.chargeResource(rebalanceType, resourceName);
-      }
-    } else {
-      // No more room for more state transitions; current states will just become intermediate
-      // states unless the partition is disabled
-      // Add this partition to a set of throttled partitions
-      for (String instance : allInstances) {
-        String currentState = currentStateMap.get(instance);
-        String bestPossibleState = bestPossibleMap.get(instance);
-        if (bestPossibleState != null && !bestPossibleState.equals(currentState)
-            && cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName())
-                .contains(instance)) {
-          // Because this partition is disabled, we allow assignment
-          intermediateMap.put(instance, bestPossibleState);
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
         } else {
-          // This partition is not disabled, so it must be throttled by just passing on the current
-          // state
-          if (currentState != null) {
-            intermediateMap.put(instance, currentState);
-          }
-          partitionsThrottled.add(partition);
+          loadMessages.add(msg);
         }
       }
     }
-    intermediatePartitionStateMap.setState(partition, intermediateMap);
   }
 
-  /**
-   * For a partition, given its preferenceList, bestPossibleState, and currentState, determine which
-   * type of rebalance is needed to model IdealState's states defined by the state model definition.
-   * @return RebalanceType needed to bring the replicas to idea states
-   *         RECOVERY_BALANCE - not all required states (replicas) are available through all
-   *         replicas, or the partition is disabled
-   *         NONE - current state matches the ideal state
-   *         LOAD_BALANCE - although all replicas required exist, Helix needs to optimize the
-   *         allocation
-   */
-  private RebalanceType getRebalanceType(ResourceControllerDataProvider cache,
-      Map<String, String> bestPossibleMap, List<String> preferenceList,
-      StateModelDefinition stateModelDef, Map<String, String> currentStateMap,
-      IdealState idealState, String partitionName) {
-    if (preferenceList == null) {
-      preferenceList = Collections.emptyList();
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+  ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -396,447 +368,386 @@ private PartitionStateMap computeIntermediatePartitionState(ResourceControllerDa
     // Perform regular load balance only if the number of partitions in recovery and in error is
     // less than the threshold. Otherwise, only allow downward-transition load balance
     boolean onlyDownwardLoadBalance = partitionCount > threshold;
-
-    loadbalanceThrottledPartitions = loadRebalance(resource, currentStateOutput,
-        bestPossiblePartitionStateMap, throttleController, intermediatePartitionStateMap,
-        partitionsNeedLoadBalance, currentStateOutput.getCurrentStateMap(resourceName),
-        onlyDownwardLoadBalance, stateModelDef, cache);
-
-    if (clusterStatusMonitor != null) {
-      clusterStatusMonitor.updateRebalancerStats(resourceName, partitionsNeedRecovery.size(),
-          partitionsNeedLoadBalance.size(), recoveryThrottledPartitions.size(),
-          loadbalanceThrottledPartitions.size());
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
     }
 
-    if (logger.isDebugEnabled()) {
-      logPartitionMapState(resourceName, new HashSet<>(resource.getPartitions()),
-          partitionsNeedRecovery, recoveryThrottledPartitions, partitionsNeedLoadBalance,
-          loadbalanceThrottledPartitions, currentStateOutput, bestPossiblePartitionStateMap,
-          intermediatePartitionStateMap);
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555402809



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(

Review comment:
       I think the gist is that you are concerned that we will spend unnecessary CPU cycles to re-count. Instead, we can do the trick of store the data and re-use them later, right? typical space and time trade off.  
   
   Changes done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r556181969



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }

Review comment:
       I dont believe have real production code for testing is a good idea.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();

Review comment:
       Same here. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482855



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
 
   @Override
   public void process(ClusterEvent event) throws Exception {
     _eventId = event.getEventId();
+
     CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
 
-    BestPossibleStateOutput bestPossibleStateOutput =
-        event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
     Map<String, Resource> resourceToRebalance =
         event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
     ResourceControllerDataProvider cache =
         event.getAttribute(AttributeName.ControllerDataProvider.name());
 
-    if (currentStateOutput == null || bestPossibleStateOutput == null || resourceToRebalance == null
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
         || cache == null) {
       throw new StageException(String.format("Missing attributes in event: %s. "
-          + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
-          event, currentStateOutput, bestPossibleStateOutput, resourceToRebalance, cache));
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",

Review comment:
       Feel free to skip the IntermediateStateCalcStage, the new logic is in PerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533105119



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -81,11 +83,15 @@ public void process(ClusterEvent event) throws Exception {
       throw new StageException(
           "Missing attributes in event:" + event + ". Requires CURRENT_STATE|RESOURCES|DataCache");
     }
+    
+    // LogUtil.logInfo(logger, _eventId, String.format("CurrentState for bestpossible is: %s", currentStateOutput));

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541348980



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -58,6 +58,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541375466



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      return selectedResourceMessages;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+    return out;
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(
+      Resource resource,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<Partition, List<Message>> selectedResourceMessages,
+

Review comment:
       done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546984789



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       I think it could be simplified into 
   if....else if ....else.
   And we don't need the nested if here. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555418251



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,

Review comment:
       moved this one to immediately after `compute`, the same as Intermediate stage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546009719



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       added TODO




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r534495523



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,951 @@
+package org.apache.helix.controller.stages;

Review comment:
       Let me copied the code to Intermedidate stage for you to have a look at the difference. This won't compile though. 
   
   That said, the code is not really copied from IntermediateStage. There is not one to one method. Be warned, there will be probably way too many differences.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541348445



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;
+    }
+    return Collections.emptyMap();
+  }
+
+  public void addResourceMessages(String resourceName, Map<Partition, List<Message>> resourceMessages) {

Review comment:
       The usage is like this.
   ```
         MessageOutput output = new MessageOutput();
   ...
         try {
           Map<Partition, List<Message>> resourceMessages =
               computePerReplicaPartitionState(idealState, currentStateOutput,
                   selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
                   bestPossibleStateOutput, dataCache,
                   throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
           output.addResourceMessages(resourceName, resourceMessages);
           retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
   ```
   
   if we remove this one, we have to expand this line `output.addResourceMessages(resourceName, resourceMessages);` to a loop, which would be less efficient. 
   
   This is pipeline core logic, saving a few cpu cycle seems to me is worthy. Let me know what is your take?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555427942



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }

Review comment:
       The removing idea theoretically also works.  
   
   Just looked at implementing this idea. Noted that `MessageOutput` is the class holding the message. However, this class does not have any removing message helper functions. It is debatable whether we want to add message removing apis to this class.
   
   Considering the benefit of using alternative approach is not really essential, maybe let us just keep the existing approach?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541406997



##########
File path: helix-core/src/test/resources/log4j.properties
##########
@@ -37,4 +39,8 @@ log4j.logger.org.apache=ERROR
 log4j.logger.com.noelios=ERROR
 log4j.logger.org.restlet=ERROR
 
+#log4j.logger.org.apache.helix.controller.stages.PerReplicaThrottleStage=INFO
+#log4j.logger.org.apache.helix.controller.stages.BestPossibleStateCalcStage=INFO
+
+#log4j.logger.org.apache.helix.integration.TestPerReplicaThrottle=INFO

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] jiajunwang closed pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
jiajunwang closed pull request #1532:
URL: https://github.com/apache/helix/pull/1532


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541482728



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/IntermediateStateCalcStage.java
##########
@@ -33,182 +32,107 @@
 import org.apache.helix.HelixException;
 import org.apache.helix.HelixManager;
 import org.apache.helix.api.config.StateTransitionThrottleConfig;
-import org.apache.helix.api.config.StateTransitionThrottleConfig.RebalanceType;
 import org.apache.helix.controller.LogUtil;
 import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
 import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
 import org.apache.helix.controller.pipeline.AbstractBaseStage;
 import org.apache.helix.controller.pipeline.StageException;
 import org.apache.helix.model.BuiltInStateModelDefinitions;
 import org.apache.helix.model.ClusterConfig;
 import org.apache.helix.model.IdealState;
 import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
 import org.apache.helix.model.Partition;
 import org.apache.helix.model.Resource;
 import org.apache.helix.model.StateModelDefinition;
-import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
-import org.apache.helix.monitoring.mbeans.ResourceMonitor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * For partition compute the Intermediate State (instance,state) pair based on the BestPossibleState
- * and CurrentState, with all constraints applied (such as state transition throttling).
- */
-public class IntermediateStateCalcStage extends AbstractBaseStage {
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
   private static final Logger logger =
-      LoggerFactory.getLogger(IntermediateStateCalcStage.class.getName());
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;

Review comment:
       The new logic is in PerReplicaThrottleStage.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541398244



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,

Review comment:
       refractored.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r553610331



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }

Review comment:
       This is for testing purpose. The test would assert the messages got throttled as load or recovery. It is not for printing purpose. See `TestPerReplicaThrottleStage`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541349066



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateOutput.java
##########
@@ -108,4 +108,5 @@ public void setPreferenceLists(String resource,
   protected boolean containsResource(String resource) {
     return _preferenceLists != null && _preferenceLists.containsKey(resource);
   }
+

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] dasahcc commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
dasahcc commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546966551



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {

Review comment:
       Name it as "out" could be confusing as naming... When I read code, I even looked back to understand what it is.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());

Review comment:
       What if it is null? Original code has a check.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(

Review comment:
       This function only serves the get expected current state. But I expect this function does the job of counting how many expected state just once.
   
   Because once we have it the expected state map, we dont need to compute again in "classifyMessage" function. It just need to do some derive, as the change already reflected from charged pending message.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);

Review comment:
       We should not count this again. Since when we charged the pending message, we need to use the retrace map. 
   
   Also, expected state does not require to recompute again as I mentioned above.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+      ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {
+      logger.error("onlyDownwardLoadBalance can't be used together with recovery_rebalance");
+      return;
+    }
+
+    String resourceName = resource.getResourceName();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    messages.sort(new MessageThrottleComparator(bestPossibleMap, currentStateMap, messagePartitionMap, stateModelDef,isRecovery));
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg: messages) {
+      if (onlyDownwardLoadBalance) {
+        boolean isDownward = isDownwardTransition(idealState, cache, msg);
+        if (isDownward == false) {
+          throttledMessages.add(msg);
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s throttled in resource as not downward: %s with type: %s", msg, resourceName,
+                    rebalanceType));
+          }
+          continue;
+        }
+      }
+
+      if (throttleController.shouldThrottleForResource(rebalanceType, resourceName)) {
+        throttledMessages.add(msg);
+        if (logger.isDebugEnabled()) {
+          LogUtil.logDebug(logger, _eventId,
+              String.format("Message: %s throttled in resource: %s with type: %s", msg, resourceName,
+                  rebalanceType));
+        }
+        continue;
+      }
+      String instance = msg.getTgtName();
+      if (throttleController.shouldThrottleForInstance(rebalanceType,instance)) {
+        throttledMessages.add(msg);
+        if (logger.isDebugEnabled()) {
+          LogUtil.logDebug(logger, _eventId,
+              String.format("Message: %s throttled in instance %s in resource: %s with type: %s", instance, msg, resourceName,
+                  rebalanceType));
+        }
+        continue;
+      }
+      throttleController.chargeInstance(rebalanceType, instance);
+      throttleController.chargeResource(rebalanceType, resourceName);
+      throttleController.chargeCluster(rebalanceType);
+      logger.trace("throttleControllerstate->{} after charge load msg: {}", throttleController, msg);
+    }
+  }
+
+   // ------------------ utilities ---------------------------
+  /**
+   * POJO that maps resource name to its priority represented by an integer.
+   */
+  private static class ResourcePriority {
+    private String _resourceName;
+    private int _priority;
+
+    ResourcePriority(String resourceName, Integer priority) {
+      _resourceName = resourceName;
+      _priority = priority;
+    }
+
+    public int compareTo(ResourcePriority resourcePriority) {
+      return Integer.compare(_priority, resourcePriority._priority);
+    }
+
+    public String getResourceName() {
+      return _resourceName;
+    }
+
+    public void setPriority(String priority) {
+      try {
+        _priority = Integer.parseInt(priority);
+      } catch (Exception e) {
+        logger.warn(
+            String.format("Invalid priority field %s for resource %s", priority, _resourceName));
+      }
+    }
+  }
+
+  private static class ResourcePriorityComparator implements Comparator<ResourcePriority> {
+    @Override
+    public int compare(ResourcePriority priority1, ResourcePriority priority2) {
+      return priority2.compareTo(priority1);
+    }
+  }
+
+  // compare message for throttling, note, all these message are of type state_transition: how about upward, downward?
+  // recovery are all upward
+  // 1) toState priority (toTop is higher than toSecond)
+  // 2) same toState, the message classification time, the less required toState meeting minActive requirement has higher priority
+  // 3) Higher priority for the partition of messages with fewer replicas with states matching with bestPossible ??? do we need this one
+  private static class MessageThrottleComparator implements Comparator<Message> {

Review comment:
       As you said, if there is no diff from this comparator and another one. Please merge them. They are very complicated. It will be hard for reader to understand the both. I will review the logic in detail once you merged.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r555418251



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,

Review comment:
       In both places, this function are the last one in the file.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle (WIP)

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r533104855



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -57,6 +57,8 @@
 import org.apache.helix.util.HelixUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import sun.rmi.runtime.Log;

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546145683



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       metrics added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] xyuanlu commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
xyuanlu commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546984789



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,956 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+
+    //TODO: add metrics
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance

Review comment:
       I think it could be simplified into 
   if....else if ....else.
   And we don't need he nested if here. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r562228681



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());

Review comment:
       Tested, this is a good point, we should have some check as original code. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r552966411



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  private void applyThrottling(Resource resource,
+      StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> currentStateMap,
+      Map<Partition, Map<String, String>> bestPossibleMap,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      boolean onlyDownwardLoadBalance,
+      List<Message> messages,
+      Map<Message, Partition> messagePartitionMap,
+      Set<Message> throttledMessages,
+      StateTransitionThrottleConfig.RebalanceType rebalanceType
+      ) {
+    boolean isRecovery = rebalanceType == StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE;
+    if (isRecovery && onlyDownwardLoadBalance) {
+      logger.error("onlyDownwardLoadBalance can't be used together with recovery_rebalance");
+      return;
+    }
+
+    String resourceName = resource.getResourceName();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    messages.sort(new MessageThrottleComparator(bestPossibleMap, currentStateMap, messagePartitionMap, stateModelDef,isRecovery));
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg: messages) {
+      if (onlyDownwardLoadBalance) {
+        boolean isDownward = isDownwardTransition(idealState, cache, msg);
+        if (isDownward == false) {

Review comment:
       fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541477541



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   */
+  private Map<Partition, List<Message>> computePerReplicaPartitionState(IdealState idealState,

Review comment:
       changed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r546145683



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,953 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  public PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THOTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+            //TODO: add metrics
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    if (dataCache.getClusterConfig().getResourcePriorityField() != null) {
+      String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        Map<Partition, List<Message>> resourceMessages =
+            computePerReplicaPartitionState(idealState, currentStateOutput,
+                selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+                bestPossibleStateOutput, dataCache,
+                throttleController, retracedPartitionsState, throttledRecoveryMsg, throttledLoadMsg);
+        output.addResourceMessages(resourceName, resourceMessages);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+

Review comment:
       added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r554258183



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -0,0 +1,982 @@
+package org.apache.helix.controller.stages;
+
+/*
+ * 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.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.helix.HelixDefinedState;
+import org.apache.helix.HelixException;
+import org.apache.helix.HelixManager;
+import org.apache.helix.api.config.StateTransitionThrottleConfig;
+import org.apache.helix.controller.LogUtil;
+import org.apache.helix.controller.common.PartitionStateMap;
+import org.apache.helix.controller.common.ResourcesStateMap;
+import org.apache.helix.controller.dataproviders.ResourceControllerDataProvider;
+import org.apache.helix.controller.pipeline.AbstractBaseStage;
+import org.apache.helix.controller.pipeline.StageException;
+import org.apache.helix.model.BuiltInStateModelDefinitions;
+import org.apache.helix.model.ClusterConfig;
+import org.apache.helix.model.IdealState;
+import org.apache.helix.model.MaintenanceSignal;
+import org.apache.helix.model.Message;
+import org.apache.helix.model.Partition;
+import org.apache.helix.model.Resource;
+import org.apache.helix.model.StateModelDefinition;
+import org.apache.helix.monitoring.mbeans.ClusterStatusMonitor;
+import org.apache.helix.monitoring.mbeans.ResourceMonitor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class PerReplicaThrottleStage extends AbstractBaseStage {
+  private static final Logger logger =
+      LoggerFactory.getLogger(PerReplicaThrottleStage.class.getName());
+
+  private boolean isEmitThrottledMsg = false;
+
+  public PerReplicaThrottleStage() {
+    this(false);
+  }
+
+  protected PerReplicaThrottleStage(boolean enableEmitThrottledMsg) {
+    isEmitThrottledMsg = enableEmitThrottledMsg;
+  }
+
+  @Override
+  public void process(ClusterEvent event) throws Exception {
+    _eventId = event.getEventId();
+
+    CurrentStateOutput currentStateOutput = event.getAttribute(AttributeName.CURRENT_STATE.name());
+
+    MessageOutput selectedMessages = event.getAttribute(AttributeName.MESSAGES_SELECTED.name());
+    LogUtil.logDebug(logger, _eventId, String.format("selectedMessages is: %s", selectedMessages));
+
+    Map<String, Resource> resourceToRebalance =
+        event.getAttribute(AttributeName.RESOURCES_TO_REBALANCE.name());
+    ResourceControllerDataProvider cache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    if (currentStateOutput == null || selectedMessages == null || resourceToRebalance == null
+        || cache == null) {
+      throw new StageException(String.format("Missing attributes in event: %s. "
+              + "Requires CURRENT_STATE (%s) |BEST_POSSIBLE_STATE (%s) |RESOURCES (%s) |DataCache (%s)",
+          event, currentStateOutput, selectedMessages, resourceToRebalance, cache));
+    }
+
+    ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
+    List<Message> throttledRecoveryMsg = new ArrayList<>();
+    List<Message> throttledLoadMsg = new ArrayList<>();
+    MessageOutput output =
+        compute(event, resourceToRebalance, currentStateOutput, selectedMessages, retracedResourceStateMap, throttledRecoveryMsg, throttledLoadMsg);
+
+    if (logger.isDebugEnabled()) {
+      LogUtil.logDebug(logger, _eventId, String.format("output is"));
+      for (String resource : resourceToRebalance.keySet()) {
+        if (output.getResourceMessages(resource) != null) {
+          LogUtil.logDebug(logger, _eventId, String.format("resource: %s", resource));
+          Map<Partition, List<Message>> partitionListMap = output.getResourceMessages(resource);
+          for (Partition partition : partitionListMap.keySet()) {
+            for (Message msg : partitionListMap.get(partition)) {
+              LogUtil.logDebug(logger, _eventId, String
+                  .format("\tresource: %s, partition: %s,  msg: %s", resource, partition, msg));
+            }
+          }
+        }
+      }
+    }
+    event.addAttribute(AttributeName.PER_REPLICA_OUTPUT_MESSAGES.name(), output);
+    LogUtil.logDebug(logger,_eventId, String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    event.addAttribute(AttributeName.PER_REPLICA_RETRACED_STATES.name(), retracedResourceStateMap);
+
+    if (isEmitThrottledMsg) {
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_RECOVERY_MESSAGES.name(), throttledRecoveryMsg);
+      event.addAttribute(AttributeName.PER_REPLICA_THROTTLED_LOAD_MESSAGES.name(), throttledLoadMsg);
+    }
+
+    // Make sure no instance has more replicas/partitions assigned than maxPartitionPerInstance. If
+    // it does, pause the rebalance and put the cluster on maintenance mode
+    int maxPartitionPerInstance = cache.getClusterConfig().getMaxPartitionsPerInstance();
+    if (maxPartitionPerInstance > 0) {
+      validateMaxPartitionsPerInstance(retracedResourceStateMap, maxPartitionPerInstance, cache, event);
+    }
+  }
+
+  /**
+   * Go through every instance in the assignment and check that each instance does NOT have more
+   * replicas for partitions assigned to it than maxPartitionsPerInstance. If the assignment
+   * violates this, put the cluster on maintenance mode.
+   * @param retracedResourceStateMap
+   * @param maxPartitionPerInstance
+   */
+  private void validateMaxPartitionsPerInstance(ResourcesStateMap retracedResourceStateMap,
+      int maxPartitionPerInstance,  ResourceControllerDataProvider cache, ClusterEvent event) {
+    Map<String, PartitionStateMap> resourceStatesMap = retracedResourceStateMap.getResourceStatesMap();
+    Map<String, Integer> instancePartitionCounts = new HashMap<>();
+
+    for (String resource : resourceStatesMap.keySet()) {
+      IdealState idealState = cache.getIdealState(resource);
+      if (idealState != null
+          && idealState.getStateModelDefRef().equals(BuiltInStateModelDefinitions.Task.name())) {
+        // Ignore task here. Task has its own throttling logic
+        continue;
+      }
+
+      PartitionStateMap partitionStateMap = resourceStatesMap.get(resource);
+      Map<Partition, Map<String, String>> stateMaps = partitionStateMap.getStateMap();
+      for (Partition p : stateMaps.keySet()) {
+        Map<String, String> stateMap = stateMaps.get(p);
+        for (String instance : stateMap.keySet()) {
+          // If this replica is in DROPPED state, do not count it in the partition count since it is
+          // to be dropped
+          String state = stateMap.get(instance);
+          if (state.equals(HelixDefinedState.DROPPED.name())) {
+            continue;
+          }
+          if (!instancePartitionCounts.containsKey(instance)) {
+            instancePartitionCounts.put(instance, 0);
+          }
+          int partitionCount = instancePartitionCounts.get(instance);
+          // Number of replicas (from different partitions) held in this instance
+          partitionCount++;
+          if (partitionCount > maxPartitionPerInstance) {
+            HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
+            String errMsg = String.format(
+                "Problem: according to this assignment, instance %s contains more "
+                    + "replicas/partitions than the maximum number allowed (%d). Pipeline will "
+                    + "stop the rebalance and put the cluster %s into maintenance mode",
+                instance, maxPartitionPerInstance, cache.getClusterName());
+            if (manager != null) {
+              if (manager.getHelixDataAccessor()
+                  .getProperty(manager.getHelixDataAccessor().keyBuilder().maintenance()) == null) {
+                manager.getClusterManagmentTool().autoEnableMaintenanceMode(
+                    manager.getClusterName(), true, errMsg,
+                    MaintenanceSignal.AutoTriggerReason.MAX_PARTITION_PER_INSTANCE_EXCEEDED);
+              }
+              LogUtil.logWarn(logger, _eventId, errMsg);
+            } else {
+              LogUtil.logError(logger, _eventId,
+                  "HelixManager is not set/null! Failed to pause this cluster/enable maintenance"
+                      + " mode due to an instance being assigned more replicas/partitions than "
+                      + "the limit.");
+            }
+
+            ClusterStatusMonitor clusterStatusMonitor =
+                event.getAttribute(AttributeName.clusterStatusMonitor.name());
+            if (clusterStatusMonitor != null) {
+              clusterStatusMonitor.setResourceRebalanceStates(Collections.singletonList(resource),
+                  ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+            }
+            throw new HelixException(errMsg);
+          }
+          instancePartitionCounts.put(instance, partitionCount);
+        }
+      }
+    }
+  }
+
+  /**
+   * Go through each resource, and based on messageSelected and currentState, compute
+   * messageOutput while maintaining throttling constraints (for example, ensure that the number
+   * of possible pending state transitions does NOT go over the set threshold).
+   * @param event
+   * @param resourceMap
+   * @param currentStateOutput
+   * @param selectedMessage
+   * @param retracedResourceStateMap out
+   * @return
+   */
+  private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap,
+      List<Message> throttledRecoveryMsg, List<Message> throttledLoadMsg) {
+    MessageOutput output = new MessageOutput();
+
+    ResourceControllerDataProvider dataCache =
+        event.getAttribute(AttributeName.ControllerDataProvider.name());
+
+    StateTransitionThrottleController throttleController =
+        new StateTransitionThrottleController(resourceMap.keySet(), dataCache.getClusterConfig(),
+            dataCache.getLiveInstances().keySet());
+
+    // Resource level prioritization based on the numerical (sortable) priority field.
+    // If the resource priority field is null/not set, the resource will be treated as lowest
+    // priority.
+    List<ResourcePriority> prioritizedResourceList = new ArrayList<>();
+    for (String resourceName : resourceMap.keySet()) {
+      prioritizedResourceList.add(new ResourcePriority(resourceName, Integer.MIN_VALUE));
+    }
+    // If resourcePriorityField is null at the cluster level, all resources will be considered equal
+    // in priority by keeping all priorities at MIN_VALUE
+    String priorityField = dataCache.getClusterConfig().getResourcePriorityField();
+    if (priorityField != null) {
+      for (ResourcePriority resourcePriority : prioritizedResourceList) {
+        String resourceName = resourcePriority.getResourceName();
+
+        // Will take the priority from ResourceConfig first
+        // If ResourceConfig does not exist or does not have this field.
+        // Try to load it from the resource's IdealState. Otherwise, keep it at the lowest priority
+        if (dataCache.getResourceConfig(resourceName) != null
+            && dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField) != null) {
+          resourcePriority.setPriority(
+              dataCache.getResourceConfig(resourceName).getSimpleConfig(priorityField));
+        } else if (dataCache.getIdealState(resourceName) != null
+            && dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField)
+            != null) {
+          resourcePriority.setPriority(
+              dataCache.getIdealState(resourceName).getRecord().getSimpleField(priorityField));
+        }
+      }
+      prioritizedResourceList.sort(new ResourcePriorityComparator());
+    }
+
+    ClusterStatusMonitor clusterStatusMonitor =
+        event.getAttribute(AttributeName.clusterStatusMonitor.name());
+    List<String> failedResources = new ArrayList<>();
+
+    // Priority is applied in assignment computation because higher priority by looping in order of
+    // decreasing priority
+    for (ResourcePriority resourcePriority : prioritizedResourceList) {
+      String resourceName = resourcePriority.getResourceName();
+
+      BestPossibleStateOutput bestPossibleStateOutput =
+          event.getAttribute(AttributeName.BEST_POSSIBLE_STATE.name());
+      if (!bestPossibleStateOutput.containsResource(resourceName)) {
+        LogUtil.logInfo(logger, _eventId, String.format(
+            "Skip calculating per replica state for resource %s because the best possible state is not available.",
+            resourceName));
+        continue;
+      }
+
+      Resource resource = resourceMap.get(resourceName);
+      IdealState idealState = dataCache.getIdealState(resourceName);
+      if (idealState == null) {
+        // If IdealState is null, use an empty one
+        LogUtil.logInfo(logger, _eventId, String
+            .format("IdealState for resource %s does not exist; resource may not exist anymore",
+                resourceName));
+        idealState = new IdealState(resourceName);
+        idealState.setStateModelDefRef(resource.getStateModelDefRef());
+      }
+
+      Map<Partition, Map<String, String>> retracedPartitionsState = new HashMap<>();
+      try {
+        throttlePerReplicaMessages(idealState, clusterStatusMonitor, currentStateOutput,
+            selectedMessage.getResourceMessages(resourceName), resourceMap.get(resourceName),
+            bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState,
+            throttledRecoveryMsg, throttledLoadMsg, output);
+        retracedResourceStateMap.setState(resourceName, retracedPartitionsState);
+      } catch (HelixException ex) {
+        LogUtil.logInfo(logger, _eventId,
+            "Failed to calculate per replica partition states for resource " + resourceName, ex);
+        failedResources.add(resourceName);
+      }
+    }
+
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.setResourceRebalanceStates(failedResources,
+          ResourceMonitor.RebalanceStatus.PER_REPLICA_STATE_CAL_FAILED);
+      clusterStatusMonitor.setResourceRebalanceStates(output.resourceSet(),
+          ResourceMonitor.RebalanceStatus.NORMAL);
+    }
+
+    return output;
+  }
+
+  /*
+   * Apply per-replica throttling logic and filter out excessive recovery and load messages for a
+   * given resource.
+   * Reconstruct retrace partition states for a resource based on pending and targeted messages
+   * Return messages for partitions of a resource.
+   * Out param retracedPartitionsCurrentState
+   * Out param output
+   */
+  private void throttlePerReplicaMessages(IdealState idealState,
+      ClusterStatusMonitor clusterStatusMonitor,
+      CurrentStateOutput currentStateOutput, Map<Partition, List<Message>> selectedResourceMessages,
+      Resource resource, BestPossibleStateOutput bestPossibleStateOutput,
+      ResourceControllerDataProvider cache, StateTransitionThrottleController throttleController,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      List<Message> throttledRecoveryMsgOut, List<Message> throttledLoadMessageOut,
+      MessageOutput output) {
+    String resourceName = resource.getResourceName();
+    LogUtil.logInfo(logger, _eventId, String.format("Processing resource: %s", resourceName));
+
+    if (!throttleController.isThrottleEnabled() || !IdealState.RebalanceMode.FULL_AUTO
+        .equals(idealState.getRebalanceMode())) {
+      retracedPartitionsStateMap.putAll(bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap());
+      for (Partition partition : selectedResourceMessages.keySet()) {
+        output.addMessages(resourceName, partition, selectedResourceMessages.get(partition));
+      }
+      return;
+    }
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+    Set<Partition> partitionsNeedRecovery = new HashSet<>();
+
+    // Step 1: charge existing pending messages and update retraced state map.
+    chargePendingMessages(resource, throttleController, currentStateOutput, bestPossibleStateOutput,
+        idealState, cache, partitionsNeedRecovery, partitionsWithErrorStateReplica,
+        retracedPartitionsStateMap);
+
+    // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    Map<Message, Partition> messagePartitionMap = new HashMap<>(); // todo: Message  may need a hashcode()
+    classifyMessages(resource, currentStateOutput, bestPossibleStateOutput, idealState, cache,
+        selectedResourceMessages, recoveryMessages, loadMessages, messagePartitionMap);
+
+    // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+
+    Map<Partition, Map<String, String>> bestPossibleMap =
+        bestPossibleStateOutput.getPartitionStateMap(resourceName).getStateMap();
+    Map<Partition, Map<String, String>> currentStateMap =
+        currentStateOutput.getCurrentStateMap(resourceName);
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, false, recoveryMessages, messagePartitionMap,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
+    // Step 4: sorts load message list and applies throttling
+
+    // TODO: consider simplifying the following logic.
+    // calculate error-on-recovery downward flag
+    // If the threshold (ErrorOrRecovery) is set, then use it, if not, then check if the old
+    // threshold (Error) is set. If the old threshold is set, use it. If not, use the default value
+    // for the new one. This is for backward-compatibility
+    int threshold = 1; // Default threshold for ErrorOrRecoveryPartitionThresholdForLoadBalance
+    int partitionCount = partitionsWithErrorStateReplica.size();
+    ClusterConfig clusterConfig = cache.getClusterConfig();
+    if (clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance() != -1) {
+      // ErrorOrRecovery is set
+      threshold = clusterConfig.getErrorOrRecoveryPartitionThresholdForLoadBalance();
+      partitionCount += partitionsNeedRecovery.size(); // Only add this count when the threshold is set
+    } else {
+      if (clusterConfig.getErrorPartitionThresholdForLoadBalance() != 0) {
+        // 0 is the default value so the old threshold has been set
+        threshold = clusterConfig.getErrorPartitionThresholdForLoadBalance();
+      }
+    }
+
+    // Perform regular load balance only if the number of partitions in recovery and in error is
+    // less than the threshold. Otherwise, only allow downward-transition load balance
+    boolean onlyDownwardLoadBalance = partitionCount > threshold;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resource, throttleController, currentStateMap, bestPossibleMap, idealState,
+        cache, onlyDownwardLoadBalance, loadMessages, messagePartitionMap, throttledLoadMessages,
+        StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled recovery message: %s", resourceName,throttledRecoveryMessages));
+    LogUtil.logDebug(logger, _eventId,
+        String.format("resource %s, throttled load messages: %s", resourceName,throttledLoadMessages));
+
+    throttledRecoveryMsgOut.addAll(throttledRecoveryMessages);
+    throttledLoadMessageOut.addAll(throttledLoadMessages);
+
+    // Step 5: construct output
+    Map<Partition, List<Message>> out = new HashMap<>();
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      List<Message> finalPartitionMessages = new ArrayList<>();
+      for (Message message: partitionMessages) {
+        if (throttledRecoveryMessages.contains(message)) {
+          continue;
+        }
+        if (throttledLoadMessages.contains(message)) {
+          continue;
+        }
+        finalPartitionMessages.add(message);
+      }
+      out.put(partition, finalPartitionMessages);
+      output.addMessages(resourceName, partition, finalPartitionMessages);
+    }
+
+    // Step 6: constructs all retraced partition state map for the resource;
+    constructRetracedPartitionStateMap(resource, retracedPartitionsStateMap, out);
+
+    // Step 7: emit metrics
+    if (clusterStatusMonitor != null) {
+      clusterStatusMonitor.updatePerReplicaRebalancerStats(resourceName, recoveryMessages.size(),
+          loadMessages.size(), throttledRecoveryMsgOut.size(),
+          throttledLoadMessageOut.size());
+    }
+  }
+
+  private void constructRetracedPartitionStateMap(Resource resource,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap,
+      Map<Partition, List<Message>> out) {
+    for (Partition partition : resource.getPartitions()) {
+      List<Message> partitionMessages = out.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+      for (Message message : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(message.getMsgType())) {
+          // todo: log?
+          // ignore cancellation message etc.
+          continue;
+        }
+        String toState = message.getToState();
+        // toIntance may not be in the retracedStateMap as so far it is current state based.
+        // new instance in best possible not in currentstate would not be in retracedStateMap yet.
+        String toInstance = message.getTgtName();
+        Map<String, String> retracedStateMap = retracedPartitionsStateMap.get(partition);
+        retracedStateMap.put(toInstance, toState);
+      }
+    }
+  }
+
+  private void getPartitionExpectedAndCurrentStateCountMap(
+      Partition partition,
+      Map<String, List<String>> preferenceLists,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Map<String, String> currentStateMap,
+      Map<String, Integer> expectedStateCountMapOut,
+      Map<String, Integer> currentStateCountsOut
+      ) {
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    int replica = idealState.getMinActiveReplicas() == -1 ? idealState
+        .getReplicaCount(preferenceList.size()) : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(cache.getEnabledLiveInstances());
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    LinkedHashMap<String, Integer> expectedStateCountMap = stateModelDef
+        .getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts
+
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet().removeAll(cache
+        .getDisabledInstancesForPartition(idealState.getResourceName(),
+            partition.getPartitionName()));
+    Map<String, Integer> currentStateCounts =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    expectedStateCountMapOut.putAll(expectedStateCountMap);
+    currentStateCountsOut.putAll(currentStateCounts);
+  }
+
+  /*
+   * Charge pending messages with recovery or load rebalance and update the retraced partition map
+   * accordingly.
+   * Also update partitionsNeedRecovery, partitionsWithErrorStateReplica accordingly which is used
+   * by later steps.
+   */
+  private void chargePendingMessages(Resource resource,
+      StateTransitionThrottleController throttleController,
+      CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState,
+      ResourceControllerDataProvider cache,
+      Set<Partition> partitionsNeedRecovery,
+      Set<Partition> partitionsWithErrorStateReplica,
+      Map<Partition, Map<String, String>> retracedPartitionsStateMap) {
+
+    logger.trace("throttleControllerstate->{} before pending message", throttleController);
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, String> retracedStateMap = new HashMap<>(currentStateMap);
+
+      if (currentStateMap.values().contains(HelixDefinedState.ERROR.name())) {
+        partitionsWithErrorStateReplica.add(partition);
+      }
+
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      Map<String, Message> pendingMessageMap =
+          currentStateOutput.getPendingMessageMap(resourceName, partition);
+      List<Message> pendingMessages = new ArrayList<>(pendingMessageMap.values());
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+      // sort pendingMessages based on transition priority then timeStamp for state transition message
+      pendingMessages.sort(new PartitionMessageComparator(stateModelDef));
+      List<Message> recoveryMessages = new ArrayList<>();
+      List<Message> loadMessages = new ArrayList<>();
+      for (Message msg : pendingMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          // ignore cancellation message etc. For now, don't charge them.
+          continue;
+        }
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+        // the gist is that if there is a topState, we should deem the topState also satisfy as secondTopState requirement.
+        // upward AND (condition 1 or condition 2)
+        // condition1: currentCount < expectedCount
+        // condition2: currentCount == expected && toState is secondary state && currentCount(topState) < expectedCount(topState)
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          partitionsNeedRecovery.add(partition);
+          // update
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+      // charge recovery message and retrace
+      for (Message recoveryMsg : recoveryMessages) {
+        String toState = recoveryMsg.getToState();
+        String toInstance = recoveryMsg.getTgtName();
+        // toInstance should be in currentStateMap
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                toInstance);
+        throttleController
+            .chargeCluster(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE,
+                resourceName);
+        logger.trace("throttleControllerstate->{} after pending recovery charge msg:{}", throttleController, recoveryMsg);
+      }
+      // charge load message and retrace;
+      // note if M->S with relay message, we don't charge relay message now. We would charge relay
+      // message only when it shows in pending messages in the next cycle of controller run.
+      for (Message loadMsg : loadMessages) {
+        String toState = loadMsg.getToState();
+        String toInstance = loadMsg.getTgtName();
+        retracedStateMap.put(toInstance, toState);
+
+        throttleController
+            .chargeInstance(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, toInstance);
+        throttleController.chargeCluster(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE);
+        throttleController
+            .chargeResource(StateTransitionThrottleConfig.RebalanceType.LOAD_BALANCE, resourceName);
+        logger.trace("throttleControllerstate->{} after pending load charge msg:{}", throttleController, loadMsg);
+      }
+      retracedPartitionsStateMap.put(partition, retracedStateMap);
+    }
+
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(Resource resource, CurrentStateOutput currentStateOutput,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Message, Partition> messagePartitionMap) {
+
+    String resourceName = resource.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    LogUtil.logInfo(logger, _eventId, String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : resource.getPartitions()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+      Map<String, Integer> expectedStateCountMap = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState,
+          cache, currentStateMap, expectedStateCountMap, currentStateCounts);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      String stateModelDefName = idealState.getStateModelDefRef();
+      StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      partitionMessages.sort(new PartitionMessageComparator(stateModelDef));
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with type %s",
+                    msg, resourceName, msg.getMsgType()));
+          }
+          continue;
+        }
+
+        messagePartitionMap.put(msg, partition);
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling
+        // we will let them pass through ASAP.
+        String instance = msg.getTgtName();
+        if (disabledInstances.contains(instance)) {
+          if (!isUpward) {
+            if (logger.isDebugEnabled()) {
+              LogUtil.logDebug(logger, _eventId,
+                  String.format("Message: %s not subject to throttle in resource: %s to disabled instancce %s",
+                      msg, resourceName, instance));
+            }
+            continue;
+          }
+        }
+
+        String toState = msg.getToState();
+        if (toState.equals(HelixDefinedState.DROPPED.name()) || toState
+            .equals(HelixDefinedState.ERROR.name())) {
+          if (logger.isDebugEnabled()) {
+            LogUtil.logDebug(logger, _eventId,
+                String.format("Message: %s not subject to throttle in resource: %s with toState %s",
+                    msg, resourceName, toState));
+          }
+          continue;
+        }
+
+        Integer expectedCount = expectedStateCountMap.get(toState);
+        Integer currentCount = currentStateCounts.get(toState);
+        expectedCount = expectedCount == null ? 0 : expectedCount;
+        currentCount = currentCount == null ? 0 : currentCount;
+
+        String topState = stateModelDef.getTopState();
+        String secondTopState = stateModelDef.getStatesPriorityList().get(1);
+        Integer expectedTopCount = expectedStateCountMap.get(topState);
+        Integer currentTopCount = currentStateCounts.get(topState);
+        currentTopCount = currentTopCount == null ? 0 : currentTopCount;
+        expectedTopCount = expectedTopCount == null ? 0 : expectedTopCount;
+
+        if (isUpward && ((currentCount < expectedCount) || (currentCount == expectedCount && toState
+            .equals(secondTopState) && currentTopCount < expectedTopCount))) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }

Review comment:
       accumulation logic is added in `getPartitionExpectedAndCurrentStateCountMap`. thus the testing logic is simplified a lot here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541349914



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/resource/ResourceMessageGenerationPhase.java
##########
@@ -30,8 +32,10 @@
 public class ResourceMessageGenerationPhase extends MessageGenerationPhase {
   @Override
   public void process(ClusterEvent event) throws Exception {
-    IntermediateStateOutput intermediateStateOutput =
-        event.getAttribute(AttributeName.INTERMEDIATE_STATE.name());
-    processEvent(event, intermediateStateOutput);
+    //IntermediateStateOutput intermediateStateOutput =

Review comment:
       removed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on pull request #1532:
URL: https://github.com/apache/helix/pull/1532#issuecomment-743690145


   @jiajunwang , thx for the review, address-ed current main logic feedback. Will add the comments about why test changed next. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org


[GitHub] [helix] kaisun2000 commented on a change in pull request #1532: Per replica throttle

Posted by GitBox <gi...@apache.org>.
kaisun2000 commented on a change in pull request #1532:
URL: https://github.com/apache/helix/pull/1532#discussion_r541234954



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/MessageOutput.java
##########
@@ -64,6 +64,18 @@ public void addMessages(String resourceName, Partition partition,
     return Collections.emptyList();
   }
 
+  public Map<Partition, List<Message>> getResourceMessages(String resourceName) {
+    Map<Partition, List<Message>> map = _messagesMap.get(resourceName);
+    if (map != null) {
+      return map;

Review comment:
       good point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org