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 2021/02/04 06:07:24 UTC

[GitHub] [helix] kaisun2000 opened a new pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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


   ### Issues
   
   - [x] My PR addresses the following Helix issues and references them in the PR description:
   
   enhance #343; second part: message classification, basic throttle logic application. Also basic test cases.
   
   ### Description
   
   - [ ] Here are some details about my PR, including screenshots of any UI changes:
   
   (Write a concise description including what, why, how)
   
   ### 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:
   
   (If CI test fails due to known issue, please specify the issue and test PR locally. Then copy & paste the result of "mvn test" to here.)
   
   ### 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] jiajunwang commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       Let me take a look and reply with a proposal. Please don't resolve the conversion with an open discussion.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId, String
+        .format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resourceName, throttleController, idealState, cache, onlyDownwardLoadBalance,
+        loadMessages, 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));

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 pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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


   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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       The original code has calculatedExpectedStateCounts and calculateCurrentStateCount in one function. Later, based on the review feedback from you and @pkuwm, we decided to separate this into two paths. That is why I would say let stick to current way and not moving back.
   
   The second point is that the code getPartitionExpectedStateCounts and getPartitionCurrentStateCounts does not have much duplication, seem to me? Do you have a proposal of how to de-dupe the duplicate logic?
   
   To be on the same page, let me list the code here. Maybe you can draw some code to illustrate the idea.
   
   ```
     private Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
         Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
         IdealState idealState, Set<String> enabledLiveInstance) {
       Map<String, Integer> expectedStateCountsOut = new HashMap<>();
   
       List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
       if (preferenceList == null) {
         preferenceList = Collections.emptyList();
       }
   
       int replica =
           idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
               : idealState.getMinActiveReplicas();
       Set<String> activeList = new HashSet<>(preferenceList);
       activeList.retainAll(enabledLiveInstance);
   
       // For each state, check that this partition currently has the required number of that state as
       // required by StateModelDefinition.
       LinkedHashMap<String, Integer> expectedStateCountMap =
           stateModelDef.getStateCountMap(activeList.size(), replica);
   
       expectedStateCountsOut.putAll(expectedStateCountMap);
       propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
   
       return expectedStateCountsOut;
     }
   ```
   
   and 
   
   ```
   private Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
         String resourceName, String partitionName, StateModelDefinition stateModelDef,
         ResourceControllerDataProvider cache) {
       // Current counts without disabled partitions or disabled instances
       Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
       currentStateMapWithoutDisabled.keySet()
           .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
       Map<String, Integer> currentStateCountsOut =
           StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
   
       propagateCountsTopDown(stateModelDef, currentStateCountsOut);
   
       return currentStateCountsOut;
     }
   
   ```
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {

Review comment:
       right.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       I think the new implementation reflects the overall design as we discussed. So I only have several comments about the code itself. But the overall direction looks fine to me.
   
   But still, please try to schedule a meeting with Junkai, who might have a better idea.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(

Review comment:
       good point. Added private.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       > Another potential flaw of this design is that we do not consider if a state transition is possible between 2 states. The current implementation implies that a lower priority state can always be transited to a higher priority state. This is not always true.
   
   This is addressed with updated `propagateCountsTopDown` implementation as discussed. The higher priority state count would be added to all the lower states that can transit to the higher priority state. 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -94,11 +100,13 @@ public void process(ClusterEvent event) throws Exception {
    * of possible pending state transitions does NOT go over the set threshold).
    * @param event
    * @param resourceMap
+   * @param currentStateOutput
    * @param selectedMessage
    * @param retracedResourceStateMap out
    */
   private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      MessageOutput selectedMessage, ResourcesStateMap retracedResourceStateMap) {
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap) {
     MessageOutput output = new MessageOutput();
 
     ResourceControllerDataProvider dataCache =

Review comment:
       See my comment bellow, there are two places using `cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());`. 
   
   We can't really get rid of it here. But I did cleaned all the path inside `throttlePerReplicaMessages` so that it won't pass `cache` around unless we need call the above 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +554,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       moved to SateModelDefinition, adopt this pattern.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +589,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       There is a logic in the PR to find all the related state for propagation, right? I was just thinking if we can let these 2 methods share some common components to reduce duplication. To be more specific, we can add a method to StateModelDefinition which return all the down stream or upper stream states given a certain state. Then the propagation count calculation can use this method to find all the propagate target, and this downward transition method can use the same method to find if the to state is one of the down stream state of the from state.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);

Review comment:
       >The names of these vars seem to be set reversely.
   The current expectedStateCountMap is really expectedStateCountByPartition, the current expectedStateCountByPartition is really the whole count map.
   
   expectedStateCountMap is for one partition.
   expectedStateCountByPartition means keyed by partition. Thus a map keyed by partition.
   
   Let me rename to the following:
   
   ```
   Map<String, Integer> partitionExpectedStateCountsOne = expectedStateCountByPartition.get(partition);
   Map<String, Integer> partitionCurrentStateCounts = currentStateCountByPartition.get(partition);
   
   ```

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);

Review comment:
       >The names of these vars seem to be set reversely.
   The current expectedStateCountMap is really expectedStateCountByPartition, the current expectedStateCountByPartition is really the whole count map.
   
   expectedStateCountMap is for one partition.
   expectedStateCountByPartition means keyed by partition. Thus a map keyed by partition.
   
   Let me rename to the following:
   
   ```
   Map<String, Integer> partitionExpectedStateCounts = expectedStateCountByPartition.get(partition);
   Map<String, Integer> partitionCurrentStateCounts = currentStateCountByPartition.get(partition);
   
   ```




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       Let me give an example to illustrate
   
   N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
   
   Assuming without the propagateCounts logic,
   
   Here, the expectedStateCountMap is {M->1, S->1, O->0}, given MIN_ACTIVE=2. 
   The currentStateCountMap is {M->0, S->1, O->0}
   
   See the message N3(O->S), the logic to test load/recovery is following:
   
   ```
   if (isUpward && (currentCount < expectedCount)) {
             recoveryMessages.add(msg);
             currentStateCounts.put(toState, currentCount + 1);
           } else {
             loadMessages.add(msg);
           }
   ```
   Here the currentCount (slave) is 1. The expectedCount is also 1.  Since current state no less than expected count for this state (SLAVE), it would classify the message as load. This is wrong of course.
   
   With population logic, we have expectedStateCountMap is {M->1, S->2 O->3}. currentStateCountMap is {M->0, S->1, O->1}. This is going to work.
   
   In theory,
   
   When determining a message as LOAD or RECOVERY,  we look at the toState of this message. If the **accumulated** current count of toState meet the required **accumulated** expected count of the toState, we will treat it as Load, otherwise, it is Recovery. Accumulation logic is implemented by `propagateCountsTopDown`. 
   Let me know if this comment make sense to you. I will add them into comment.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);

Review comment:
       No. See my comments as why we need propagateCountsTopDown.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       I was looking back to this previous comment because I remember that I suggest that we put only "basic throttle" logic in this PR. But it seems not the 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {

Review comment:
       right. let me change the name to `stateCountMap`




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       As discussed offline, here is reasoning about first case as agreed.
   
   The first issue raised can be illustrated use this example:
   ```
   Preference list N0, N1, N2
   Current N0 - O, N1 - S, N2 - S
   current count  M(0), S(2), O (2)
   Message
         N0 O -> S (Load)
   
   ```
   The N0 (0->S) is classified as load in this case. Since eventually N0 will be master based on PreferenceList. So should we give it preference as recovery? 
   
   For now, we decide not do this kind of optimization for the reason that classifying it as load won't delay a master being promoted to N1. Thus, eventually N0 would be promoted to Master once N0 arrive at slave state. The gist is best possible would step-by-step lead to preference list. For now, we can rely on best possible stage to make the choice. We will note this potential optimization in design doc.
   
   
   
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -66,12 +70,11 @@ public void process(ClusterEvent event) throws Exception {
     }
 
     ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
-    MessageOutput output =
-        compute(event, resourceToRebalance, selectedMessages, retracedResourceStateMap);
+    MessageOutput output = compute(event, resourceToRebalance, currentStateOutput, selectedMessages,
+        retracedResourceStateMap);
 
     event.addAttribute(AttributeName.PER_REPLICA_THROTTLE_OUTPUT_MESSAGES.name(), output);
-    LogUtil.logDebug(logger, _eventId,
-        String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    logger.debug("Event info: {}, retraceResourceStateMap is: {}", retracedResourceStateMap);

Review comment:
       All changed to this way.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +589,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       moved to StateModelDefinition.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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) {

Review comment:
       Will change to use two different private methods.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -66,12 +70,11 @@ public void process(ClusterEvent event) throws Exception {
     }
 
     ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
-    MessageOutput output =
-        compute(event, resourceToRebalance, selectedMessages, retracedResourceStateMap);
+    MessageOutput output = compute(event, resourceToRebalance, currentStateOutput, selectedMessages,
+        retracedResourceStateMap);
 
     event.addAttribute(AttributeName.PER_REPLICA_THROTTLE_OUTPUT_MESSAGES.name(), output);
-    LogUtil.logDebug(logger, _eventId,
-        String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    logger.debug("Event info: {}, retraceResourceStateMap is: {}", retracedResourceStateMap);

Review comment:
       Better to keep the format `Event {} :  retraceResourceStateMap is: {}`
   Missing the event id.
   
   As I mentioned before, I'd prefer to make the string format consistent with the `LogUtil`. Otherwise, keeping the LogUtil and use `if (debug on)` for debug level for performance consideration.
   
   

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {

Review comment:
       +1. A for loop is preferred: index var will be a local/smaller scope variable in the for loop.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +554,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       Just an example, I'd design the API like this concisely:
   ```
   private boolean isDownwardTransition(Map<String, Integer> statePriorityMap, String fromState, String toState) {
      Integer fromStatePriority = statePriorityMap.get(fromState);
      Integer toStatePriority = statePriorityMap.get(toState);
      return fromStatePriority != null && toStatePriority != null && fromStatePriority < toStatePriority;
   }
   ```




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -146,8 +153,8 @@ private MessageOutput compute(ClusterEvent event, Map<String, Resource> resource
           partitonMsgMap.put(partition, msgList);
         }
         MessageOutput resourceMsgOut =
-            throttlePerReplicaMessages(idealState, partitonMsgMap, bestPossibleStateOutput,
-                throttleController, retracedPartitionsState);
+            throttlePerReplicaMessages(idealState, currentStateOutput, partitonMsgMap,
+                bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState);

Review comment:
       We can't really get rid of it here. But I did cleaned all the path inside throttlePerReplicaMessages so that it won't pass cache around unless we need call the above 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }

Review comment:
       changed to 
   
   `List<String> preferenceList =
           preferenceLists.getOrDefault(partition.getPartitionName(), Collections.emptyList());`




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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

Review comment:
       I guess this was intentional, from intermedidateStage. Let me remove "// StateModelDefinition's counts". No value of this comment as it is obvious anyway.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -66,12 +70,11 @@ public void process(ClusterEvent event) throws Exception {
     }
 
     ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
-    MessageOutput output =
-        compute(event, resourceToRebalance, selectedMessages, retracedResourceStateMap);
+    MessageOutput output = compute(event, resourceToRebalance, currentStateOutput, selectedMessages,
+        retracedResourceStateMap);
 
     event.addAttribute(AttributeName.PER_REPLICA_THROTTLE_OUTPUT_MESSAGES.name(), output);
-    LogUtil.logDebug(logger, _eventId,
-        String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    logger.debug("Event info: {}, retraceResourceStateMap is: {}", retracedResourceStateMap);

Review comment:
       All changed to `logger.isDebugEnable() { LogUtil.logDebug(...)}`




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +554,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       Based on previous discussion, will move this one to stateModelDefinition. 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);

Review comment:
       Where is the change, please?




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));

Review comment:
       Now I see your point. Will change. Nowadays, with auto boxing, we don't need to do it this way explicitly. 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {

Review comment:
       changed to for loop.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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


   


-- 
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {

Review comment:
       
   See the following, this logic is working for FULL_AUTO only.  Is this going to be a problem? Let me sync with you offline.
   
   ```
   private MessageOutput throttlePerReplicaMessages
   
       // TODO: expand per-replica-throttling beyond FULL_AUTO
       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 output;
       }
   
   ```




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;

Review comment:
       changed and moved this part to StateModelDefinition.
   
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       Let me give an example to illustrate
   
   N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
   
   Assuming without the propagateCounts logic,
   
   Here, the expectedStateCountMap is {M->1, S->1, O->0}, given MIN_ACTIVE=2. 
   The currentStateCountMap is {M->0, S->1, O->0}
   
   See the message N3(O->S), the logic to test load/recovery is following:
   
   ```
   if (isUpward && (currentCount < expectedCount)) {
             recoveryMessages.add(msg);
             currentStateCounts.put(toState, currentCount + 1);
           } else {
             loadMessages.add(msg);
           }
   ```
   Here the currentCount (slave) is 1. The expectedCount is also 1.  Since current state no less than expected count for this state (SLAVE), it would classify the message as load. This is wrong of course.
   
   With population logic, we have expectedStateCountMap is {M->1, S->2 O->3}. currentStateCountMap is {M->0, S->1, O->1}. This is going to work.
   
   In theory,
   
   When determining a message as LOAD or RECOVERY,  we look at the toState of this message. If the toState current count meet the required expected count of the toState, we will treat it as Load, otherwise, it is 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -146,8 +153,8 @@ private MessageOutput compute(ClusterEvent event, Map<String, Resource> resource
           partitonMsgMap.put(partition, msgList);
         }
         MessageOutput resourceMsgOut =
-            throttlePerReplicaMessages(idealState, partitonMsgMap, bestPossibleStateOutput,
-                throttleController, retracedPartitionsState);
+            throttlePerReplicaMessages(idealState, currentStateOutput, partitonMsgMap,
+                bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState);

Review comment:
       let me see how long the parameter list would be.
   
   cache.getStateModelDef(stateModelDefName);
   cache.getEnabledLiveInstances()
   
   This one is deadly, as we need to get per partition. We have to have cache in throttlePerReplicaMessages.
   
   cache
           .getDisabledInstancesForPartition(idealState.getResourceName(),
               partition.getPartitionName())
   
   Let me reduce passing cache to some private method called by `throttlePerReplicaMessages` if the don't call getDisabledInstancesForPartition . But I can't get rid of cache here. Some private method do need 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, IdealState idealState,
+      ResourceControllerDataProvider cache, boolean onlyDownwardLoadBalance, List<Message> messages,
+      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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(idealState, cache, msg)) {
+          throttledMessages.add(msg);
+          if (logger.isDebugEnabled()) {

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next

Review comment:
       changed description.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -146,8 +153,8 @@ private MessageOutput compute(ClusterEvent event, Map<String, Resource> resource
           partitonMsgMap.put(partition, msgList);
         }
         MessageOutput resourceMsgOut =
-            throttlePerReplicaMessages(idealState, partitonMsgMap, bestPossibleStateOutput,
-                throttleController, retracedPartitionsState);
+            throttlePerReplicaMessages(idealState, currentStateOutput, partitonMsgMap,
+                bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState);

Review comment:
       Follow up on what I said above, we shall be able to send the resource-related information only here. A whole cache object is not necessary. I understand that it may not be able to execute ideally, but please have a try. Let me know if there is any blocker.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -94,11 +100,13 @@ public void process(ClusterEvent event) throws Exception {
    * of possible pending state transitions does NOT go over the set threshold).
    * @param event
    * @param resourceMap
+   * @param currentStateOutput
    * @param selectedMessage
    * @param retracedResourceStateMap out
    */
   private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      MessageOutput selectedMessage, ResourcesStateMap retracedResourceStateMap) {
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap) {
     MessageOutput output = new MessageOutput();
 
     ResourceControllerDataProvider dataCache =

Review comment:
       It is one of the headaches in our legacy code that we tend to pass a huge object to the private method and let it fetch whatever it needs. This is in general against the good design pattern. Can we refine it here so that the following private method only gets whatever parameters it wants but nothing in addition?
   
   I know it will make the parameter list longer. But I strongly prefer setting up a very clear parameter list. This leads to a good module design.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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) {

Review comment:
       There are 2 different works done here. Maybe splitting into 2 methods is a better plan.
   
   Also, another thing that is minor but I really dislike is that the 2 output parameter names are not symmetric... One is ***MapnOut another one is just ***Out. Why is that?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);

Review comment:
       Why do we need to change currentStateCounts because of the classifying?
   
   I mean I know the reason. But please add a comment here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);
+
+        // for disabled disabled instance, the downward transition is not subjected to load throttling

Review comment:
       nit, but this kind of comment really prevents the reviewers to understand your intention. Please revisit and fix here and if more comments like this.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);

Review comment:
       Will this work?
   
   >     LinkedHashMap<String, Integer> expectedStateCountMap =
   >         stateModelDef.getStateCountMap(activeList.size(), replica); // StateModelDefinition's counts

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {

Review comment:
       The first sentence of the related issue is, "MIN_ACTIVE replica is only applied in DelayAutoRebalancer and not respected by throttling logic in Helix." So I think is logic is not what we want eventually. Are you planning to add it in a later PR? If so, then let's add a TODO here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {

Review comment:
       It is also used for the current state count map, right?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       Sorry that I don't really understand what is the functionality of this method. Could you please add more comments?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);

Review comment:
       Why we need to understand the state model for the current state counts?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       Hasn't we decided to put only "basic throttle" logic in this PR? The DownwardTransition shall be put in a later PR, right?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();

Review comment:
       I guess you want to use an ordered map here? Or what's the point of referring to the state model and read state priority in the later logic?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);

Review comment:
       The names of these vars seem to be set reversely.
   The current expectedStateCountMap is really expectedStateCountByPartition, the current expectedStateCountByPartition is really the whole count 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] kaisun2000 commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +539,24 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(IdealState idealState, ResourceControllerDataProvider cache,
+      Message message) {
+    boolean isDownward = false;
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    Map<String, Integer> statePriorityMap = stateModelDef.getStatePriorityMap();
+    String fromState = message.getFromState();
+    String toState = message.getToState();
+    if (statePriorityMap.containsKey(fromState) && statePriorityMap.containsKey(toState)) {

Review comment:
       change to following comment:
   
   // Only when both fromState and toState can be found in the statePriorityMap, comparision of
       // state priority is used to determine if the transition is downward. Otherwise, we can't
       // really determine the order and by default consider the transition not downard.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);

Review comment:
       see my comment in `propagateCountsTopDown`. We can discuss there.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));

Review comment:
       changed and moved this part to StateModelDefinition.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +589,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       StateModelDefinition.isDownwardTransition(message), just like that.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       I remember that conversation, I get what you said. But still, I personally prefer keeping the util. For 2 reasons, 1. it enforces everyone to put the event Id. 2. we can add whatever additional information or output these logs to somewhere else whenever required. If we all write different codes everywhere, then it is not possible.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //
+        if (isUpward && (currentCount < minimumRequiredCount)) {
+          recoveryMessages.add(msg);
+          // It is critical to increase toState value by one here. For example, current state
+          // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
+          // In this case, the first O->S would be classified as recovery. Then this line would
+          // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
+          // would be correctly marked as load message.
+          partitionCurrentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, StateModelDefinition stateModelDef,
+      boolean onlyDownwardLoadBalance, List<Message> messages, 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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(stateModelDef, msg)) {

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       Is it possible that `StatePriorityList` is not set in the state model and so stateList is returned null?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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) {

Review comment:
       A function having 2 output params just looks unclean to me :) It'd be better if we can figure out a cleaner method.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);

Review comment:
       `Integer` -> `int`

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap

Review comment:
       Just a question: for index = 0, prevState is the same as curState. Is it correct?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }

Review comment:
       `getOrDefault()`?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,

Review comment:
       8 params look too many to me. I would simplify it if possible. There are too many logic branches inside the method, based on  `onlyDownwardLoadBalance` and `rebalanceType`, which could also conflict when `isRecovery && onlyDownwardLoadBalance`. Might not be a good design. Not sure if we could split it into two methods for load rebalance and recover rebalance, or upward and downward..
   
   Roughly looking, `throttledMessages` is the output, why not put it as a return?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }

Review comment:
       Nit, can be simplified with a preferred method `expectedStateCountMap.putIfAbsent(prevState, 0);`

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId, String
+        .format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resourceName, throttleController, idealState, cache, onlyDownwardLoadBalance,
+        loadMessages, 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));

Review comment:
       logDebug is not protected by `if debugEnabled`, which would cause unexpected string creation overhead.
   Please add `if debugEnabled`, or use `logger.debug();`, so strings won't be created/formatted at info/error logging level. 
   
   Same for other `logDebug` in this PR.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, IdealState idealState,
+      ResourceControllerDataProvider cache, boolean onlyDownwardLoadBalance, List<Message> messages,
+      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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(idealState, cache, msg)) {
+          throttledMessages.add(msg);
+          if (logger.isDebugEnabled()) {

Review comment:
       The debug code is kind of distracting. If they are really necessary, I would suggest changing them to `logger.debug()` which could reduce lines of distracting code.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;

Review comment:
       `map.getOrDefault()` will help.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);

Review comment:
       `List<String> stateList = stateModelDef.getStatesPriorityList();` 
   inside, we need to loop from top state to lower state.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/test/java/org/apache/helix/controller/stages/TestPerReplicaThrottleStage.java
##########
@@ -100,6 +115,246 @@ public void testNoThrottleMessagePass() {
     Assert.assertTrue(msg.getId().equals("001"));
   }
 
+  // case 0. N1(O), N2(S), N3(O), message N3(O->S) is treated as recovery

Review comment:
       added as `testOneRecoveryOneLoad`.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       If you were referring to the very first version, then the original method is called "getPartitionExpectedAndCurrentStateCountMap". That method is definitely NOT what I'm trying to suggest you to do. That was one single method, but you were just putting all the logic together.
   
   For this new code, what I thought is actually avoid duplicating the loops in the calculateXXXXCount() methods. However, you will have to pre-calculate the input with a loop for expected state map and current state map separately anyway, so it is not possible to save here.
   
   The only possible way is to abstract the pre-calculate logic into other classes and then make it uniform calls. This requires a larger scope refactoring and may still not make the cut. So obviously, the cost is much higher than the gain. Let's just keep the current implemenation. I can help to have a try to see if it can be refined, later once this PR is in.
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       changed to pattern 
   
   ```
   if (logger.isDebugEnabled()) {
               LogUtil.logDebug(logger, _eventId, String.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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.

Review comment:
       Let me move this logic to StateModelDefinition.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,

Review comment:
       You are right. Now it is broken to two functions The typo does not exists anymore.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));

Review comment:
       value in the statePriorityMap is Integer, not int. 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;

Review comment:
       Let me move this logic to StateModelDefinition as utility function.
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       I did removed dropped partition and error partition handling.
   
    Removed one more part
    ```
         Set<String> disabledInstances =
             cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
   ```
   So the code is now like this:
   ```
         for (Message msg : partitionMessages) {
           if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
             logger.debug(
                 "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
                 _eventId, msg, resourceName, msg.getMsgType());
             continue;
           }
   
           boolean isUpward = !isDownwardTransition(stateModelDef, msg);
   
           // TODO: add disabled disabled instance special treatment
   
           String toState = msg.getToState();
   
           // TODO: dropped and Error state special treatment
   
           Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
           Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
   
           //
           if (isUpward && (currentCount < minimumRequiredCount)) {
             recoveryMessages.add(msg);
             // It is critical to increase toState value by one here. For example, current state
             // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
             // In this case, the first O->S would be classified as recovery. Then this line would
             // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
             // would be correctly marked as load message.
             partitionCurrentStateCounts.put(toState, currentCount + 1);
           } else {
             loadMessages.add(msg);
           }
         }
   
   ```
   
   Let me know if there is anything else? Or we can comment on the new 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] dasahcc commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;

Review comment:
       I believe state model should have the function to get priority of states. Even if it does not have it, this computation logic should be there.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {

Review comment:
       It is not safe to use while (true). The only condition is index < stateList.size(), then let's put it here.
   
   Or we can just use for loop to lopp state list.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));

Review comment:
       So the rank is int, why we need Integer.valueOf(rank)?




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);

Review comment:
       No. See my comments as why we need propagateCountsTopDown. I leave that open for discussion.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +56,10 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    if (logger.isDebugEnabled()) {

Review comment:
       Do we want to move this check into the LogUtil? Or we want to save one function call for extreme performance? I personally prefer cleaner code. But no strong preference here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       If you were referring to the very first version, then the original method is called "getPartitionExpectedAndCurrentStateCountMap". That method is definitely NOT what I'm trying to suggest you to do. That was one single method, but you were just putting all the logic together.
   
   For this new code, what I thought is actually avoid duplicating the loops in the calculateXXXXCount() methods. However, as you also mentioned, you will have to pre-calculate the input with a loop for expected state map and current state map separately anyway, so it is not possible to save here.
   
   The only possible way is to abstract the pre-calculate logic into other classes and then make it uniform calls. This requires a larger scope refactoring and may still not make the cut. So obviously, the cost is much higher than the gain. Let's just keep the current implemenation. I can help to have a try to see if it can be refined, later once this PR is in.
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       ping




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,

Review comment:
       let me sync with you offline. break would have a lot of 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();

Review comment:
       This is why we have the populateTopDownLogic. Let me add some comment, see if it make sense to you. Or we can sync offline.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       Think about if there is specially defined statemodel which has no statelist. That does mean it is necessarily wrong. It is just that there is not state count  top down calculation needed, right? Let us don't throw exception.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);

Review comment:
       see my comment in `propagateCountsTopDown`




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       The original code has calculatedExpectedStateCounts and calculateCurrentStateCount in one function. Later, based on the review feedback from you and @pkuwm, we decided to separate this into two paths. That is why I would say let stick to current way and not moving back.
   
   The second point is that the code getPartitionExpectedStateCounts and getPartitionCurrentStateCounts does not have much duplication, seem to me? Do you have a proposal of how to de-dupe the duplicate logic. For sure I can use the new approach.
   
   To be on the same page, let me list the code here. Maybe you can draw some code to illustrate the idea.
   
   ```
     private Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
         Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
         IdealState idealState, Set<String> enabledLiveInstance) {
       Map<String, Integer> expectedStateCountsOut = new HashMap<>();
   
       List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
       if (preferenceList == null) {
         preferenceList = Collections.emptyList();
       }
   
       int replica =
           idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
               : idealState.getMinActiveReplicas();
       Set<String> activeList = new HashSet<>(preferenceList);
       activeList.retainAll(enabledLiveInstance);
   
       // For each state, check that this partition currently has the required number of that state as
       // required by StateModelDefinition.
       LinkedHashMap<String, Integer> expectedStateCountMap =
           stateModelDef.getStateCountMap(activeList.size(), replica);
   
       expectedStateCountsOut.putAll(expectedStateCountMap);
       propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
   
       return expectedStateCountsOut;
     }
   ```
   
   and 
   
   ```
   private Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
         String resourceName, String partitionName, StateModelDefinition stateModelDef,
         ResourceControllerDataProvider cache) {
       // Current counts without disabled partitions or disabled instances
       Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
       currentStateMapWithoutDisabled.keySet()
           .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
       Map<String, Integer> currentStateCountsOut =
           StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
   
       propagateCountsTopDown(stateModelDef, currentStateCountsOut);
   
       return currentStateCountsOut;
     }
   
   ```
   




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       I was looking back to this previous comment because I remember that I suggest that we put only "basic throttle" logic in this PR. And we agreed on this point. But it seems not the 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //
+        if (isUpward && (currentCount < minimumRequiredCount)) {
+          recoveryMessages.add(msg);
+          // It is critical to increase toState value by one here. For example, current state
+          // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
+          // In this case, the first O->S would be classified as recovery. Then this line would
+          // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
+          // would be correctly marked as load message.
+          partitionCurrentStateCounts.put(toState, currentCount + 1);

Review comment:
       Yes, I think so. 
   
   We can reason it this way:
   The propagated current state counts only reflected what current state is. This line is more like saying, if this new message delivered and the new state change happens, what it will be. 
   
   Let me what is your take 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       LogUtil was used before. @pkuwm pointed out that there is runtime inefficiency of LogUtil. He prefers to use raw logger which does not have this inefficiency. That is why all of them changed to logger.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);

Review comment:
       This is critical. Let me add a comment 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -66,12 +70,11 @@ public void process(ClusterEvent event) throws Exception {
     }
 
     ResourcesStateMap retracedResourceStateMap = new ResourcesStateMap();
-    MessageOutput output =
-        compute(event, resourceToRebalance, selectedMessages, retracedResourceStateMap);
+    MessageOutput output = compute(event, resourceToRebalance, currentStateOutput, selectedMessages,
+        retracedResourceStateMap);
 
     event.addAttribute(AttributeName.PER_REPLICA_THROTTLE_OUTPUT_MESSAGES.name(), output);
-    LogUtil.logDebug(logger, _eventId,
-        String.format("retraceResourceStateMap is: %s", retracedResourceStateMap));
+    logger.debug("Event info: {}, retraceResourceStateMap is: {}", retracedResourceStateMap);

Review comment:
       All changed `logger.isDebugEnable() { LogUtil.logDebug(...)}`




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap

Review comment:
       right. The idea is to add the index 0 count to index 1 count and then to index2 count ...




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();

Review comment:
       comment added. We can discuss in that thread. Resolve 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] kaisun2000 commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       changed to `if ( stateList == null || stateList.size() <= 0) {`. 
   
   Properly constructed stateModelDef should not have null StatePriorityList.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       Think about if there is specially defined statemodel which has no statelist. That does mean it is necessarily wrong. It is just that there is not state count  top down calculation needed, right? 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        boolean isUpward = !isDownwardTransition(idealState, cache, msg);

Review comment:
       removed. special treatment of like this would be in next PRs




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       Is it possible that `StatePriorityList` is not set in the state model and so stateList is returned null?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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) {

Review comment:
       A function having 2 output params just looks unclean to me :) It'd be better if we can figure out a cleaner method.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);

Review comment:
       `Integer` -> `int`

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap

Review comment:
       Just a question: for index = 0, prevState is the same as curState. Is it correct?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }

Review comment:
       `getOrDefault()`?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,

Review comment:
       8 params look too many to me. I would simplify it if possible. There are too many logic branches inside the method, based on  `onlyDownwardLoadBalance` and `rebalanceType`, which could also conflict when `isRecovery && onlyDownwardLoadBalance`. Might not be a good design. Not sure if we could split it into two methods for load rebalance and recover rebalance, or upward and downward..
   
   Roughly looking, `throttledMessages` is the output, why not put it as a return?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }

Review comment:
       Nit, can be simplified with a preferred method `expectedStateCountMap.putIfAbsent(prevState, 0);`

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;
+    Set<Message> throttledLoadMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId, String
+        .format("applying load rebalance with resource %s, onlyDownwardLoadBalance %s",
+            resourceName, onlyDownwardLoadBalance));
+    applyThrottling(resourceName, throttleController, idealState, cache, onlyDownwardLoadBalance,
+        loadMessages, 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));

Review comment:
       logDebug is not protected by `if debugEnabled`, which would cause unexpected string creation overhead.
   Please add `if debugEnabled`, or use `logger.debug();`, so strings won't be created/formatted at info/error logging level. 
   
   Same for other `logDebug` in this PR.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, IdealState idealState,
+      ResourceControllerDataProvider cache, boolean onlyDownwardLoadBalance, List<Message> messages,
+      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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(idealState, cache, msg)) {
+          throttledMessages.add(msg);
+          if (logger.isDebugEnabled()) {

Review comment:
       The debug code is kind of distracting. If they are really necessary, I would suggest changing them to `logger.debug()` which could reduce lines of distracting code.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;

Review comment:
       `map.getOrDefault()` will help.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -94,11 +100,13 @@ public void process(ClusterEvent event) throws Exception {
    * of possible pending state transitions does NOT go over the set threshold).
    * @param event
    * @param resourceMap
+   * @param currentStateOutput
    * @param selectedMessage
    * @param retracedResourceStateMap out
    */
   private MessageOutput compute(ClusterEvent event, Map<String, Resource> resourceMap,
-      MessageOutput selectedMessage, ResourcesStateMap retracedResourceStateMap) {
+      CurrentStateOutput currentStateOutput, MessageOutput selectedMessage,
+      ResourcesStateMap retracedResourceStateMap) {
     MessageOutput output = new MessageOutput();
 
     ResourceControllerDataProvider dataCache =

Review comment:
       See my comment bellow, there are two places using `cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());`. 
   
   We can't really get rid of it here. But I did cleaned all the path inside `throttlePerReplicaMessages` that won't pass cache around unless we need call the above call.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;

Review comment:
       changed to
    `        Integer expectedCount = expectedStateCountMap.getOrDefault(toState, 0);
           Integer currentCount = currentStateCounts.getOrDefault(toState, 0);




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +202,52 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition =
+        calculateExpectedStateCounts(selectedResourceMessages, bestPossibleStateOutput, idealState,
+            cache.getEnabledLiveInstances(), stateModelDef);
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition =
+        calculateCurrentStateCount(selectedResourceMessages, currentStateOutput, stateModelDef,
+            resourceName, cache);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+
+    classifyMessages(resourceName, stateModelDef, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    logger.debug("Event info {}, applying recovery rebalance with resource {}", _eventId,
+        resourceName);
+    applyThrottling(resourceName, throttleController, stateModelDef, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;

Review comment:
       default to false.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +589,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       Let me make sure I understand.
   
   1/ " feel you can use some variance of this method to calculate the propagate state count map"? This is to test if a message is downward. Seems to me, totally irrelevant to calculate the propagate state count map. Can you elaborate? Maybe some code. I totally don't get it.
   
   2/ For sure I can move this one to stateModel definition. 
   
   Let me sync with your over slack.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       Ok, for now let me resolve this issue. 




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +539,24 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(IdealState idealState, ResourceControllerDataProvider cache,
+      Message message) {
+    boolean isDownward = false;
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+    Map<String, Integer> statePriorityMap = stateModelDef.getStatePriorityMap();
+    String fromState = message.getFromState();
+    String toState = message.getToState();
+    if (statePriorityMap.containsKey(fromState) && statePriorityMap.containsKey(toState)) {

Review comment:
       Please clean up this section. You changed the code from previous Intermediate stage, but not comment. They don't match now.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,

Review comment:
       Why this function is named as `ExistingAndCurrent`. Do you mean `expected`?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next

Review comment:
       I saw a lot of TODOs. How do you justify the correctness of this PR? I'm confused what can be considered as `basic` throttle. Can you have more detailed explanation in PR description part? I'd like to see which are nice-to-have, which are must-have but not included in this PR.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, IdealState idealState,
+      ResourceControllerDataProvider cache, boolean onlyDownwardLoadBalance, List<Message> messages,
+      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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(idealState, cache, msg)) {
+          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()) {

Review comment:
       We now have logs for the total number of partitions that get throttled for different rebalance types before. Do you have that kind of general log information besides in debug mode?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {

Review comment:
       Then shouldn't you throw an exception in case of null as an alert? 

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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

Review comment:
       Please reformat all the code in the editor and clean up comments.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -208,7 +254,12 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       }
       List<Message> finalPartitionMessages = new ArrayList<>();
       for (Message message : partitionMessages) {
-        // TODO: next PR messages exclusion
+        if (throttledRecoveryMessages.contains(message)) {

Review comment:
       nit: You can just combine these two conditions for simplicity. 

##########
File path: helix-core/src/test/java/org/apache/helix/controller/stages/TestPerReplicaThrottleStage.java
##########
@@ -100,6 +115,246 @@ public void testNoThrottleMessagePass() {
     Assert.assertTrue(msg.getId().equals("001"));
   }
 
+  // case 0. N1(O), N2(S), N3(O), message N3(O->S) is treated as recovery

Review comment:
       I was thinking a test case that can validate the following scenario:
   Current states: N1(S), N2(O), N3(O)
   BestPossible: N1(M), N2(S), N3(S)
   Only one message is considered as recovery rebalance based on your defined priority rule. 
   Do you think this can be testified in this PR? You current test cases do not seem to be testing 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] dasahcc commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));

Review comment:
       But it is not necessary to explicitly 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] jiajunwang commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       Why not using the LogUtil? It generates a uniform log format with event id, so I think the original one is better. But please let me know if you have any other concerns here.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.

Review comment:
       This logic can be moved to StateModelDefinition as a util method there.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //
+        if (isUpward && (currentCount < minimumRequiredCount)) {
+          recoveryMessages.add(msg);
+          // It is critical to increase toState value by one here. For example, current state
+          // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
+          // In this case, the first O->S would be classified as recovery. Then this line would
+          // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
+          // would be correctly marked as load message.
+          partitionCurrentStateCounts.put(toState, currentCount + 1);

Review comment:
       Do we need to propagate the count here too? Since the original input count was calculated with propagation, right?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       I see there are other changes like this in the PR. I will not comment in the other places. But please reply here to help me understand. Thanks

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,

Review comment:
       It seems that the only difference between getPartitionExpectedStateCounts and getPartitionCurrentStateCounts is that you need to calculate the expected state count map from the preference list. Is it possible to move this logic outside to calculateExpectedStateCounts so that we can avoid some duplicate code?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -270,4 +589,23 @@ public void setPriority(String priority) {
       }
     }
   }
+
+  private boolean isDownwardTransition(StateModelDefinition stateModelDef, Message message) {

Review comment:
       I feel you can use some variance of this method to calculate the propagate state count map. If this is the case, we can move this method to the state model definition (or a util class) to cleanup the code here. More importantly, the class will provide better leverage.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(

Review comment:
       Are the following several methods intentionally added as package-private?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //

Review comment:
       ?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //
+        if (isUpward && (currentCount < minimumRequiredCount)) {
+          recoveryMessages.add(msg);
+          // It is critical to increase toState value by one here. For example, current state
+          // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
+          // In this case, the first O->S would be classified as recovery. Then this line would
+          // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
+          // would be correctly marked as load message.
+          partitionCurrentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, StateModelDefinition stateModelDef,
+      boolean onlyDownwardLoadBalance, List<Message> messages, 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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(stateModelDef, msg)) {

Review comment:
       nit, if (onlyDownwardLoadBalance && !isDownwardTransition(stateModelDef, msg))... for simplicity.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +202,52 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition =
+        calculateExpectedStateCounts(selectedResourceMessages, bestPossibleStateOutput, idealState,
+            cache.getEnabledLiveInstances(), stateModelDef);
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition =
+        calculateCurrentStateCount(selectedResourceMessages, currentStateOutput, stateModelDef,
+            resourceName, cache);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+
+    classifyMessages(resourceName, stateModelDef, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    logger.debug("Event info {}, applying recovery rebalance with resource {}", _eventId,
+        resourceName);
+    applyThrottling(resourceName, throttleController, stateModelDef, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;

Review comment:
       Since this will be TODO, shall we just ignore this boolean in this PR, or hardcode it to be false?
   I'm concerned to put the "1" as a hardcoded threshold here. It is confusing and might be forgotten to remove 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -208,7 +254,12 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       }
       List<Message> finalPartitionMessages = new ArrayList<>();
       for (Message message : partitionMessages) {
-        // TODO: next PR messages exclusion
+        if (throttledRecoveryMessages.contains(message)) {

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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +202,52 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    String stateModelDefName = idealState.getStateModelDefRef();
+    StateModelDefinition stateModelDef = cache.getStateModelDef(stateModelDefName);
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition =
+        calculateExpectedStateCounts(selectedResourceMessages, bestPossibleStateOutput, idealState,
+            cache.getEnabledLiveInstances(), stateModelDef);
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition =
+        calculateCurrentStateCount(selectedResourceMessages, currentStateOutput, stateModelDef,
+            resourceName, cache);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+
+    classifyMessages(resourceName, stateModelDef, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    logger.debug("Event info {}, applying recovery rebalance with resource {}", _eventId,
+        resourceName);
+    applyThrottling(resourceName, throttleController, stateModelDef, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next
+    // TODO: this can be done together with chargePendingMessage() where partitionsNeedRecovery is from
+    boolean onlyDownwardLoadBalance = partitionsWithErrorStateReplica.size() > 1;

Review comment:
       changed to false for 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] kaisun2000 commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {
+          recoveryMessages.add(msg);
+          currentStateCounts.put(toState, currentCount + 1);
+        } else {
+          loadMessages.add(msg);
+        }
+      }
+    }
+  }
+
+  protected void applyThrottling(String resourceName,
+      StateTransitionThrottleController throttleController, IdealState idealState,
+      ResourceControllerDataProvider cache, boolean onlyDownwardLoadBalance, List<Message> messages,
+      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;
+    }
+
+    // TODO: add message sorting in next PR
+    logger.trace("throttleControllerstate->{} before load", throttleController);
+    for (Message msg : messages) {
+      if (onlyDownwardLoadBalance) {
+        if (!isDownwardTransition(idealState, cache, msg)) {
+          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()) {

Review comment:
       A little bit confused. let me sync with you offline.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -146,8 +153,8 @@ private MessageOutput compute(ClusterEvent event, Map<String, Resource> resource
           partitonMsgMap.put(partition, msgList);
         }
         MessageOutput resourceMsgOut =
-            throttlePerReplicaMessages(idealState, partitonMsgMap, bestPossibleStateOutput,
-                throttleController, retracedPartitionsState);
+            throttlePerReplicaMessages(idealState, currentStateOutput, partitonMsgMap,
+                bestPossibleStateOutput, dataCache, throttleController, retracedPartitionsState);

Review comment:
       let me see how long the parameter list would be.
   
   cache.getStateModelDef(stateModelDefName);
   cache.getEnabledLiveInstances()
   
   This one is deadly, as we need to get per partition. We have to have cache in throttlePerReplicaMessages.
   
   cache
           .getDisabledInstancesForPartition(idealState.getResourceName(),
               partition.getPartitionName())
   
   Let reduce passing cache to some private method called by `throttlePerReplicaMessages` if the don't call getDisabledInstancesForPartition . But I can't get rid of cache here. Some private method do need 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,239 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   */
+
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    stateCountMap.putIfAbsent(prevState, 0);
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      int prevCnt = stateCountMap.get(prevState);
+      stateCountMap.put(curState, prevCnt + stateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() == -1 ? idealState.getReplicaCount(preferenceList.size())

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] jiajunwang commented on pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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


   I guess the core of this PR is how we calculate the expected replica count and how we use it to categorize the message. I have some questions regarding the design, already comment on the PR. But I think it would be more efficient if you can help to host a meeting for the stakeholders to discuss 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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);

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] jiajunwang commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,274 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   *
+   *  Note, there can be customerized model that having more than one route to a top state. For
+   *  example S1, S2, S3 are three levels of states with S1 as top state with lowest priority.
+   *  It is possible that S2 can transits upward to S1 while S3 can also transits upward to S1.
+   *  Thus, we consider S1 meet count requirement of both S2 and S3. In propagation time, we will
+   *  add state count of S1 to both S2 and S3.
+   */
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+
+    Map<String, Integer> statePriorityMap = new HashMap<>();
+
+    // calculate rank of each state. Next use the rank to compare if a transition is upward or not
+    int rank = 0;
+    for (String state : stateList) {
+      statePriorityMap.put(state, Integer.valueOf(rank));
+      rank++;
+    }
+
+    // given a key state, find the set of states that can transit to this key state upwards.
+    Map<String, Set<String>> fromStatesMap = new HashMap<>();
+    for (String transition : stateModelDef.getStateTransitionPriorityList()) {
+      // Note, we assume stateModelDef is properly constructed.
+      String[] fromStateAndToState = transition.split("-");
+      String fromState = fromStateAndToState[0];
+      String toState = fromStateAndToState[1];
+      Integer fromStatePriority = statePriorityMap.get(fromState);
+      Integer toStatePriority = statePriorityMap.get(toState);
+      if (fromStatePriority.compareTo(toStatePriority) <= 0) {
+        // skip downward transitition
+        continue;
+      }
+      fromStatesMap.putIfAbsent(toState, new HashSet<>());
+      fromStatesMap.get(toState).add(fromState);
+    }
+
+    // propagation by adding state counts of current state to all lower priority state that can
+    // transit to this current state
+    int index = 0;
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      stateCountMap.putIfAbsent(curState, 0);
+      Integer curCount = stateCountMap.get(curState);
+      // for all states S that can transition to curState, add curState count back to S in stateCountMap
+      for (String fromState : fromStatesMap.getOrDefault(curState, Collections.emptySet())) {
+        Integer fromStateCount = stateCountMap.getOrDefault(fromState, 0);
+        stateCountMap.put(fromState, Integer.sum(fromStateCount, curCount));
+      }
+      index++;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() <= 0 ? idealState.getReplicaCount(preferenceList.size())
+            : idealState.getMinActiveReplicas();
+    Set<String> activeList = new HashSet<>(preferenceList);
+    activeList.retainAll(enabledLiveInstance);
+
+    // For each state, check that this partition currently has the required number of that state as
+    // required by StateModelDefinition.
+    LinkedHashMap<String, Integer> expectedStateCountMap =
+        stateModelDef.getStateCountMap(activeList.size(), replica);
+
+    expectedStateCountsOut.putAll(expectedStateCountMap);
+    propagateCountsTopDown(stateModelDef, expectedStateCountsOut);
+
+    return expectedStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateExpectedStateCounts(
+      Map<Partition, List<Message>> selectedResourceMessages,
+      BestPossibleStateOutput bestPossibleStateOutput, IdealState idealState,
+      Set<String> enabledLiveInstance, StateModelDefinition stateModelDef) {
+    Map<Partition, Map<String, Integer>> expectedStateCountsByPartition = new HashMap<>();
+
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCounts =
+          getPartitionExpectedStateCounts(partition, preferenceLists, stateModelDef, idealState,
+              enabledLiveInstance);
+      expectedStateCountsByPartition.put(partition, expectedStateCounts);
+    }
+
+    return expectedStateCountsByPartition;
+  }
+
+  Map<String, Integer> getPartitionCurrentStateCounts(Map<String, String> currentStateMap,
+      String resourceName, String partitionName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache) {
+    // Current counts without disabled partitions or disabled instances
+    Map<String, String> currentStateMapWithoutDisabled = new HashMap<>(currentStateMap);
+    currentStateMapWithoutDisabled.keySet()
+        .removeAll(cache.getDisabledInstancesForPartition(resourceName, partitionName));
+    Map<String, Integer> currentStateCountsOut =
+        StateModelDefinition.getStateCounts(currentStateMapWithoutDisabled);
+
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+
+    return currentStateCountsOut;
+  }
+
+  Map<Partition, Map<String, Integer>> calculateCurrentStateCount(
+      Map<Partition, List<Message>> selectedResourceMessages, CurrentStateOutput currentStateOutput,
+      StateModelDefinition stateModelDef, String resourceName,
+      ResourceControllerDataProvider cache) {
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> currentStateCounts =
+          getPartitionCurrentStateCounts(currentStateMap, resourceName,
+              partition.getPartitionName(), stateModelDef, cache);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+
+    return currentStateCountsByPartition;
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, StateModelDefinition stateModelDef,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    logger.info("Event info {}, Classify message for resource {} ", _eventId, resourceName);
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> partitionExpectedStateCounts =
+          expectedStateCountByPartition.get(partition);
+      Map<String, Integer> partitionCurrentStateCounts =
+          currentStateCountsByPartition.get(partition);
+
+      List<Message> partitionMessages = selectedResourceMessages.get(partition);
+      if (partitionMessages == null) {
+        continue;
+      }
+
+      // sort partitionMessages based on transition priority and then creation timestamp for transition message
+      // TODO: sort messages in same partition in next PR
+      Set<String> disabledInstances =
+          cache.getDisabledInstancesForPartition(resourceName, partition.getPartitionName());
+      for (Message msg : partitionMessages) {
+        if (!Message.MessageType.STATE_TRANSITION.name().equals(msg.getMsgType())) {
+          logger.debug(
+              "Event info: {} Message: {} not subject to throttle in resource: {} with type {}",
+              _eventId, msg, resourceName, msg.getMsgType());
+          continue;
+        }
+
+        boolean isUpward = !isDownwardTransition(stateModelDef, msg);
+
+        // TODO: add disabled disabled instance special treatment
+
+        String toState = msg.getToState();
+
+        // TODO: dropped and Error state special treatment
+
+        Integer minimumRequiredCount = partitionExpectedStateCounts.getOrDefault(toState, 0);
+        Integer currentCount = partitionCurrentStateCounts.getOrDefault(toState, 0);
+
+        //
+        if (isUpward && (currentCount < minimumRequiredCount)) {
+          recoveryMessages.add(msg);
+          // It is critical to increase toState value by one here. For example, current state
+          // of 3 replica in a partition is (M, O, O). Two messages here bringing up the two O to S.
+          // In this case, the first O->S would be classified as recovery. Then this line would
+          // increase the S state in partitionCurrentStateCounts value by one. Next O->S message
+          // would be correctly marked as load message.
+          partitionCurrentStateCounts.put(toState, currentCount + 1);

Review comment:
       I mean simply add one to the immediate toState is not propagation, right? Unless I take something wrong, but I think you should add all the further to states down the line 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] jiajunwang commented on a change in pull request #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,239 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  /*
+   *  Let us use an example to illustrate the state count accumulation idea:
+   *  Currentstate N1(O), N2(S), N3(O), message N3(O->S) should be classified as recovery.
+   *  Assuming without the propagateCounts logic,
+   *  Here, the expectedStateCounts is {M->1, S->1, O->0}, given MIN_ACTIVE=2.
+   *  The currentStateCounts is {M->0, S->1, O->0}.
+   *  At the time, message N3(O->S) is tested for load/recovery, the logic would compare
+   *  currentStateCounts[S] with expectedStateCounts[S]. If less than expected, it is deemed as
+   *  recovery; otherwise load.
+   *  Then the message would be incorrectly classified as load.
+   *
+   *  With propogationTopDown, we have expectedStateCounts as {M->1, S->2 O->3}.
+   *  currentStateCountCounts as {M->0, S->1, O->1}. Thus the message is going to be classified
+   *  as recovery correctly.
+   *
+   *  The gist is that:
+   *  When determining a message as LOAD or RECOVERY, we look at the toState of this message.
+   *  If the accumulated current count of toState meet the required accumulated expected count
+   *  of the toState, we will treat it as Load, otherwise, it is Recovery.
+   */
+
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> stateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList == null || stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    stateCountMap.putIfAbsent(prevState, 0);
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      int prevCnt = stateCountMap.get(prevState);
+      stateCountMap.put(curState, prevCnt + stateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  Map<String, Integer> getPartitionExpectedStateCounts(Partition partition,
+      Map<String, List<String>> preferenceLists, StateModelDefinition stateModelDef,
+      IdealState idealState, Set<String> enabledLiveInstance) {
+    Map<String, Integer> expectedStateCountsOut = new HashMap<>();
+
+    List<String> preferenceList = preferenceLists.get(partition.getPartitionName());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    int replica =
+        idealState.getMinActiveReplicas() == -1 ? idealState.getReplicaCount(preferenceList.size())

Review comment:
       To make the code more robust, shall we change idealState.getMinActiveReplicas() == -1 to idealState.getMinActiveReplicas() <=0 ?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,
+      Map<String, Integer> expectedStateCountMap) {
+    // attribute state in higher priority to lower priority
+    List<String> stateList = stateModelDef.getStatesPriorityList();
+    if (stateList.size() <= 0) {
+      return;
+    }
+    int index = 0;
+    String prevState = stateList.get(index);
+    if (!expectedStateCountMap.containsKey(prevState)) {
+      expectedStateCountMap.put(prevState, 0);
+    }
+    while (true) {
+      if (index == stateList.size() - 1) {
+        break;
+      }
+      index++;
+      String curState = stateList.get(index);
+      String num = stateModelDef.getNumInstancesPerState(curState);
+      if ("-1".equals(num)) {
+        break;
+      }
+      Integer prevCnt = expectedStateCountMap.get(prevState);
+      expectedStateCountMap
+          .put(curState, prevCnt + expectedStateCountMap.getOrDefault(curState, 0));
+      prevState = curState;
+    }
+  }
+
+  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());
+    if (preferenceList == null) {
+      preferenceList = Collections.emptyList();
+    }
+
+    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);
+    propagateCountsTopDown(stateModelDef, expectedStateCountMapOut);
+    propagateCountsTopDown(stateModelDef, currentStateCountsOut);
+  }
+
+  void calculateExistingAndCurrentStateCount(Map<Partition, List<Message>> selectedResourceMessages,
+      CurrentStateOutput currentStateOutput, BestPossibleStateOutput bestPossibleStateOutput,
+      IdealState idealState, ResourceControllerDataProvider cache,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    String resourceName = idealState.getResourceName();
+    Map<String, List<String>> preferenceLists =
+        bestPossibleStateOutput.getPreferenceLists(resourceName);
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, String> currentStateMap =
+          currentStateOutput.getCurrentStateMap(resourceName, partition);
+
+      Map<String, Integer> expectedStateCounts = new HashMap<>();
+      Map<String, Integer> currentStateCounts = new HashMap<>();
+      getPartitionExpectedAndCurrentStateCountMap(partition, preferenceLists, idealState, cache,
+          currentStateMap, expectedStateCounts, currentStateCounts);
+
+      // save these two maps for later usage
+      expectedStateCountByPartition.put(partition, expectedStateCounts);
+      currentStateCountsByPartition.put(partition, currentStateCounts);
+    }
+  }
+
+  /*
+   * Classify the messages of each partition into recovery and load messages.
+   */
+  private void classifyMessages(String resourceName, IdealState idealState,
+      ResourceControllerDataProvider cache, Map<Partition, List<Message>> selectedResourceMessages,
+      List<Message> recoveryMessages, List<Message> loadMessages,
+      Map<Partition, Map<String, Integer>> expectedStateCountByPartition,
+      Map<Partition, Map<String, Integer>> currentStateCountsByPartition) {
+    LogUtil.logInfo(logger, _eventId,
+        String.format("Classify message for resource: %s", resourceName));
+
+    for (Partition partition : selectedResourceMessages.keySet()) {
+      Map<String, Integer> expectedStateCountMap = expectedStateCountByPartition.get(partition);
+      Map<String, Integer> currentStateCounts = currentStateCountsByPartition.get(partition);
+
+      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
+      // TODO: sort messages in same partition in next PR
+      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;
+        }
+
+        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;
+
+        if (isUpward && (currentCount < expectedCount)) {

Review comment:
       Synced offline. I think the logic is correct here, given the "expectedCount" is derived based on the MIN_ACTIVE_REPLICA_COUNT. Can we change the name to something like "minimumRequiredCount"?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       Another potential flaw of this design is that we do not consider if a state transition is possible between 2 states. The current implementation implies that a lower priority state can always be transited to a higher priority state. This is not always true.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -220,6 +271,224 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
     return output;
   }
 
+  private void propagateCountsTopDown(StateModelDefinition stateModelDef,

Review comment:
       This is a nice design, but does it cover this case, current state {M>0, S>2, 0>1}, and the message is O->S. The expectation is that the message will be done and then the S->M is the next one. However, it will be treated as loadbalance, right?
   
   It seems to be a very difficult problem. But I think we need to address it because the original algorithm will generate the right result.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -195,11 +204,48 @@ private MessageOutput throttlePerReplicaMessages(IdealState idealState,
       return output;
     }
 
-    // TODO: later PRs
+    Set<Partition> partitionsWithErrorStateReplica = new HashSet<>();
+
+    Map<Partition, Map<String, Integer>> expectedStateCountByPartition = new HashMap<>();
+    Map<Partition, Map<String, Integer>> currentStateCountsByPartition = new HashMap<>();
+
+    calculateExistingAndCurrentStateCount(selectedResourceMessages, currentStateOutput,
+        bestPossibleStateOutput, idealState, cache, expectedStateCountByPartition,
+        currentStateCountsByPartition);
+
     // Step 1: charge existing pending messages and update retraced state map.
+    // TODO: later PRs
     // Step 2: classify all the messages into recovery message list and load message list
+    List<Message> recoveryMessages = new ArrayList<>();
+    List<Message> loadMessages = new ArrayList<>();
+    classifyMessages(resourceName, idealState, cache, selectedResourceMessages, recoveryMessages,
+        loadMessages, expectedStateCountByPartition, currentStateCountsByPartition);
+
     // Step 3: sorts recovery message list and applies throttling
+    Set<Message> throttledRecoveryMessages = new HashSet<>();
+    LogUtil.logDebug(logger, _eventId,
+        String.format("applying recovery rebalance with resource %s", resourceName));
+    applyThrottling(resourceName, throttleController, idealState, cache, false, recoveryMessages,
+        throttledRecoveryMessages, StateTransitionThrottleConfig.RebalanceType.RECOVERY_BALANCE);
+
     // Step 4: sorts load message list and applies throttling
+    // TODO: calculate error-on-recovery downward threshold with complex backward compatibility next

Review comment:
       Basic throttle is what we have now, namely classifying the messages to load/recovery, throttle accordingly.
   More features are 1/ pending message accounting 2/ Error or recovery with relationship to downward only 3/ disabled partition handling 4/ dropped and error message handling. 
   
   None of them are nice to have. They all must to have. We will add them in the subsequent PRs.




----------------------------------------------------------------
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 #1628: Per Replica Throttle -- 2nd: messages classification and basic throttle application

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/PerReplicaThrottleStage.java
##########
@@ -51,7 +55,7 @@ public void process(ClusterEvent event) throws Exception {
     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));
+    logger.debug("event info: {}, selectedMessages is: {}", _eventId, selectedMessages);

Review comment:
       I remember that conversation, and I think Huizhi is right. But to solve the concern, why not modifying the LogUtil to fix the inefficiency? I checked the code, 8 lines of change. However, if we discard this util, then there would be much more code to change. Anyway, we don't need to include the util change in this PR. But please keep using the util and we can fix it in a separate PR.




----------------------------------------------------------------
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