You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/09/02 16:35:16 UTC

[GitHub] [kafka] C0urante commented on a diff in pull request #12561: KAFKA-12495: Exponential backoff retry to prevent rebalance storms when worker joins after revoking rebalance

C0urante commented on code in PR #12561:
URL: https://github.com/apache/kafka/pull/12561#discussion_r961848839


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java:
##########
@@ -307,9 +309,31 @@ ClusterAssignment performTaskAssignment(
                     existing.tasks().addAll(assignment.tasks());
                 }
             );
-            canRevoke = toExplicitlyRevoke.size() == 0;
+
+            // If this round and the previous round involved revocation, we will do an exponential
+            // backoff delay to prevent rebalance storms.
+            if (revokedInPrevious && !toExplicitlyRevoke.isEmpty()) {
+                numSuccessiveRevokingRebalances++;
+                delay = (int) consecutiveRevokingRebalancesBackoff.backoff(numSuccessiveRevokingRebalances);
+                log.debug("Consecutive revoking rebalances observed. Need to wait for {} ms", delay);

Review Comment:
   By gating the delay behind `revokedInPrevious`, we're always allowing the first round that would require allocation-balancing revocations to actually include those revocations. The proposal I made should only cause us to introduce the delay (and gate revocations behind that delay) after the second consecutive rebalance that would require allocation-balancing revocations.
   
   I'd also like to reiterate this point:
   > we should only issue an assignment with a delay when that delay is the time until the re-allocation of lost assignments and/or the next round where we will permit allocation-balancing revocations.
   
   I agree that if we feel comfortable permitting revocations in a given round, then we should just go ahead and include those revocations in the assignment--but in that case, we don't need to include a delay in the assignment, especially since workers will all automatically rejoin the cluster after they've revoked the appropriate connectors/tasks.



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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

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