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

[GitHub] [helix] jiajunwang opened a new pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

jiajunwang opened a new pull request #1360:
URL: https://github.com/apache/helix/pull/1360


   ### Issues
   
   - [X] My PR addresses the following Helix issues and references them in the PR description:
   
   Resolves #1349 
   
   ### Description
   
   - [X] Here are some details about my PR, including screenshots of any UI changes:
   
   This PR depends on #1359 , please reviewing that PR first.
   
   If no rebalancer class name is specified, use the DelayedAutoRebalancer instead of AutoRebalancer.
   Note that DelayedAutoRebalancer is compatible with the AutoRebalanceStrategy. So the assignment would remain the same even after migration.
   However, since DelayedAutoRebalancer keeps a cached rebalance result in the Controller cache, it causes a behavior change.
   - We need to modify multiple test classes to use the CrushEd rebalance strategy for the correct test logic.
   - This behavior change only impacts BestPossibleExternalViewVerifier if the resource was on the older AutoRebalancer (by DEFAULT setting) and AutoRebalanceStrategy.
   - If any Helix client is using BestPossibleExternalViewVerifier with the described configuration, then changing the rebalancer class config in the IdealState explicitly to AutoRebalancer is required before the Helix version bump-up. Or the BestPossibleExternalViewVerifier may fail.
   
   ### Tests
   
   - [X] The following tests are written for this issue:
   
   Multiple related tests have been updated to cover the new change.
   
   - [X] The following is the result of the "mvn test" command on the appropriate module:
   
   [INFO] Tests run: 1200, Failures: 0, Errors: 0, Skipped: 0, Time elapsed: 4,453.792 s - in TestSuite
   [INFO]
   [INFO] Results:
   [INFO]
   [INFO] Tests run: 1200, Failures: 0, Errors: 0, Skipped: 0
   [INFO]
   [INFO] ------------------------------------------------------------------------
   [INFO] BUILD SUCCESS
   [INFO] ------------------------------------------------------------------------
   [INFO] Total time: 01:14 h
   [INFO] Finished at: 2020-09-10T19:52:49-07:00
   [INFO] ------------------------------------------------------------------------
   
   ### Documentation (Optional)
   
   - In case of new functionality, my PR adds documentation in the following wiki page:
   
   (Link the GitHub wiki you added)
   
   ### Commits
   
   - My commits all reference appropriate Apache Helix GitHub issues in their subject lines. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
     1. Subject is separated from body by a blank line
     1. Subject is limited to 50 characters (not including Jira issue reference)
     1. Subject does not end with a period
     1. Subject uses the imperative mood ("add", not "adding")
     1. Body wraps at 72 characters
     1. Body explains "what" and "why", not "how"
   
   ### Code Quality
   
   - My diff has been formatted using helix-style.xml 
   (helix-style-intellij.xml if IntelliJ IDE is used)
   


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

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



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


[GitHub] [helix] kaisun2000 commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -347,36 +347,39 @@ private boolean computeSingleResourceBestPossibleState(ClusterEvent event,
       LogUtil.logError(logger, _eventId, "Error computing assignment for resource " + resourceName
           + ". no rebalancer found. rebalancer: " + rebalancer + " mappingCalculator: "
           + mappingCalculator);
-    }
-
-    if (rebalancer != null && mappingCalculator != null) {
-      ResourceAssignment partitionStateAssignment = null;
+    } else {
       try {
         HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
         rebalancer.init(manager);
         idealState =
             rebalancer.computeNewIdealState(resourceName, idealState, currentStateOutput, cache);
 
-        output.setPreferenceLists(resourceName, idealState.getPreferenceLists());
+        // Check if calculation is done successfully
+        if (!checkBestPossibleStateCalculation(idealState)) {
+          LogUtil.logWarn(logger, _eventId,
+              "The calculated idealState is not valid, resource: " + resourceName);
+          return false;
+        }
 
         // Use the internal MappingCalculator interface to compute the final assignment
         // The next release will support rebalancers that compute the mapping from start to finish
-        partitionStateAssignment = mappingCalculator
+        ResourceAssignment partitionStateAssignment = mappingCalculator
             .computeBestPossiblePartitionState(cache, idealState, resource, currentStateOutput);
 
         if (partitionStateAssignment == null) {
           LogUtil.logWarn(logger, _eventId,
-              "PartitionStateAssignment is null, resource: " + resourceName);
+              "The calculated partitionStateAssignment is null, resource: " + resourceName);
           return false;
         }
 
+        // Set the calculated result to the output.
+        output.setPreferenceLists(resourceName, idealState.getPreferenceLists());
         for (Partition partition : resource.getPartitions()) {
           Map<String, String> newStateMap = partitionStateAssignment.getReplicaMap(partition);
           output.setState(resourceName, partition, newStateMap);
         }
 
-        // Check if calculation is done successfully
-        return checkBestPossibleStateCalculation(idealState);

Review comment:
       The changes in this method is the same #1359., right? Basically without it, a empty preference list can be set in 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] jiajunwang merged pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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


   


----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       The argument here is that we do want to deprecate AutoRebalancer and AutoRebalanceStrategy. Since they are buggy and unstable. So even the behavior change will hurt some of their use cases, we will take it.
   If this assumption does not hold, then we can never change the default rebalancer to DelayedAutoRebalancer.




----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       Synced offline with Kai. It is clear now.




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

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



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


[GitHub] [helix] jiajunwang commented on pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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


   This PR has been approved and re-tested after rebase to the master branch. It is ready to be merged.


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

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



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


[GitHub] [helix] jiajunwang commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestRebalancerMetrics.java
##########
@@ -128,8 +128,10 @@ public void testLoadBalanceMetrics() {
     event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput);
     setupLiveInstances(4);
 
-    runStage(event, new ReadClusterDataStage());

Review comment:
       It is moved to later. Since we do need to clear the cache before running this stage.




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

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



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


[GitHub] [helix] kaisun2000 commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
        So controller cache only works for DelayedAutoRebalancer and AutoRebalanceStrategy? I am not sure exactly where it is. Can you point to a code link?
   




----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       Synced offline with Kai. It is clear now.




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

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



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


[GitHub] [helix] jiajunwang merged pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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


   


----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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


   This PR has been approved and re-tested after rebase to the master branch. It is ready to be merged.


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

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



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


[GitHub] [helix] kaisun2000 commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -118,18 +120,25 @@ public void beforeMethod() throws IOException {
 
   @Test
   public void testParticipantUnavailable() throws Exception {
-    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
+    IdealState idealState = new FullAutoModeISBuilder(testDb)

Review comment:
       Let us make sure I understand this test fix correctly. With #1359 fix, the net effect is that previous missing "rebalance error" metric is added. Also, avoid setting empty best possible state to the `output` which is used by later rebalance stage. 
   
   In this test, we did not assert anything about rebalance error metrics; thus, even missing one rebalance failure metrics, it would cause this test not stable. So why do we need the change in this test?
   
   Put it another way, what caused the this test unstable?

##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       Previously, we use AutoRebalancer with AutoRebalanceStrategy. With this change, we use DelayedAutoRebalancer with CrushEd strategy.
   
   The question is that now, by default with this change, we will use DelayedAutoRebalancer by default. If we forget to add CrushEdRebalanceStrategy, the default seems to be AutoRebalanceStrategy. Will that work?

##########
File path: helix-core/src/test/java/org/apache/helix/monitoring/mbeans/TestRebalancerMetrics.java
##########
@@ -128,8 +128,10 @@ public void testLoadBalanceMetrics() {
     event.addAttribute(AttributeName.CURRENT_STATE.name(), currentStateOutput);
     setupLiveInstances(4);
 
-    runStage(event, new ReadClusterDataStage());

Review comment:
       why remove this line?

##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -118,18 +120,25 @@ public void beforeMethod() throws IOException {
 
   @Test
   public void testParticipantUnavailable() throws Exception {
-    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
+    IdealState idealState = new FullAutoModeISBuilder(testDb)

Review comment:
       I see, this assertion may not work, if pipeline run another cycle?
   
   ```
       pollForError(accessor, errorNodeKey);
       checkRebalanceFailureGauge(true);
       checkResourceBestPossibleCalFailureState(
           ResourceMonitor.RebalanceStatus.BEST_POSSIBLE_STATE_CAL_FAILED, testDb);
   
   ```




----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       The argument here is that we do want to deprecate AutoRebalancer and AutoRebalanceStrategy. Since they are buggy and unstable. So even the behavior change will hurt some of their use cases (any logic that use the bestpossible verifier), we will take it.
   If this assumption does not hold, then we can never change the default rebalancer to DelayedAutoRebalancer.




----------------------------------------------------------------
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 #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -118,18 +120,25 @@ public void beforeMethod() throws IOException {
 
   @Test
   public void testParticipantUnavailable() throws Exception {
-    _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());
+    IdealState idealState = new FullAutoModeISBuilder(testDb)

Review comment:
       Out of date. Let's focus on the newer change.




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

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



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


[GitHub] [helix] kaisun2000 commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/main/java/org/apache/helix/controller/stages/BestPossibleStateCalcStage.java
##########
@@ -347,36 +347,39 @@ private boolean computeSingleResourceBestPossibleState(ClusterEvent event,
       LogUtil.logError(logger, _eventId, "Error computing assignment for resource " + resourceName
           + ". no rebalancer found. rebalancer: " + rebalancer + " mappingCalculator: "
           + mappingCalculator);
-    }
-
-    if (rebalancer != null && mappingCalculator != null) {
-      ResourceAssignment partitionStateAssignment = null;
+    } else {
       try {
         HelixManager manager = event.getAttribute(AttributeName.helixmanager.name());
         rebalancer.init(manager);
         idealState =
             rebalancer.computeNewIdealState(resourceName, idealState, currentStateOutput, cache);
 
-        output.setPreferenceLists(resourceName, idealState.getPreferenceLists());
+        // Check if calculation is done successfully
+        if (!checkBestPossibleStateCalculation(idealState)) {
+          LogUtil.logWarn(logger, _eventId,
+              "The calculated idealState is not valid, resource: " + resourceName);
+          return false;
+        }
 
         // Use the internal MappingCalculator interface to compute the final assignment
         // The next release will support rebalancers that compute the mapping from start to finish
-        partitionStateAssignment = mappingCalculator
+        ResourceAssignment partitionStateAssignment = mappingCalculator
             .computeBestPossiblePartitionState(cache, idealState, resource, currentStateOutput);
 
         if (partitionStateAssignment == null) {
           LogUtil.logWarn(logger, _eventId,
-              "PartitionStateAssignment is null, resource: " + resourceName);
+              "The calculated partitionStateAssignment is null, resource: " + resourceName);
           return false;
         }
 
+        // Set the calculated result to the output.
+        output.setPreferenceLists(resourceName, idealState.getPreferenceLists());
         for (Partition partition : resource.getPartitions()) {
           Map<String, String> newStateMap = partitionStateAssignment.getReplicaMap(partition);
           output.setState(resourceName, partition, newStateMap);
         }
 
-        // Check if calculation is done successfully
-        return checkBestPossibleStateCalculation(idealState);

Review comment:
       The changes in this method is the same #1359., right? Basically without it, a empty preference list can be set in 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] jiajunwang commented on a change in pull request #1360: Use the DelayedAutoRebalancer as the default rebalancer.

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



##########
File path: helix-core/src/test/java/org/apache/helix/integration/TestAlertingRebalancerFailure.java
##########
@@ -161,7 +170,8 @@ public void testParticipantUnavailable() throws Exception {
   @Test(dependsOnMethods = "testParticipantUnavailable")
   public void testTagSetIncorrect() throws Exception {
     _gSetupTool.addResourceToCluster(CLUSTER_NAME, testDb, 5,
-        BuiltInStateModelDefinitions.MasterSlave.name(), RebalanceMode.FULL_AUTO.name());

Review comment:
       It works in production. But it does not work in the test.
   The reason is that our verifier does not read the controller cache (which is only used by the DelayedAutoRebalancer).
   So the controller only does one calculation, and later pipeline will only read from the cache. But the verifier is calculating from scratch every time. And since the AutoRebalanceStrategy is not stable, the verify result will never converge.
   If we change to CrushEd, then even the verifier computing from scratch, the result would be the same. And our tests can pass.




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