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


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java:
##########
@@ -286,30 +294,43 @@ ClusterAssignment performTaskAssignment(
         handleLostAssignments(lostAssignments, newSubmissions, 
completeWorkerAssignment);
 
         // Do not revoke resources for re-assignment while a delayed rebalance 
is active
-        // Also we do not revoke in two consecutive rebalances by the same 
leader
-        canRevoke = delay == 0 && canRevoke;
-
-        // Compute the connectors-and-tasks to be revoked for load balancing 
without taking into
-        // account the deleted ones.
-        log.debug("Can leader revoke tasks in this assignment? {} (delay: 
{})", canRevoke, delay);
-        if (canRevoke) {
+        if (delay == 0) {
             Map<String, ConnectorsAndTasks> toExplicitlyRevoke =
-                    performTaskRevocation(activeAssignments, 
currentWorkerAssignment);
-
-            log.debug("Connector and task to revoke assignments: {}", 
toRevoke);
-
-            toExplicitlyRevoke.forEach(
-                (worker, assignment) -> {
-                    ConnectorsAndTasks existing = toRevoke.computeIfAbsent(
-                        worker,
-                        v -> new ConnectorsAndTasks.Builder().build());
-                    existing.connectors().addAll(assignment.connectors());
-                    existing.tasks().addAll(assignment.tasks());
+                    performTaskRevocation(configured, 
completeWorkerAssignment);
+
+            // If this round and the previous round involved revocation, we 
will calculate a delay for
+            // the next round when revoking rebalance would be allowed. Note 
that delay could be 0, in which
+            // case we would always revoke.
+            if (revokedInPrevious && !toExplicitlyRevoke.isEmpty()) {
+                numSuccessiveRevokingRebalances++;
+                log.debug("Consecutive revoking rebalances observed. Computing 
delay and next scheduled rebalance.");
+                delay = (int) 
consecutiveRevokingRebalancesBackoff.backoff(numSuccessiveRevokingRebalances);
+                if (delay != 0) {
+                    scheduledRebalance = time.milliseconds() + delay;
+                    log.debug("Skipping revocations in the current round with 
a delay of {}ms. Next scheduled rebalance:{}",
+                            delay, scheduledRebalance);
+                } else {
+                    log.debug("Revoking assignments as 
scheduled.rebalance.max.delay.ms is set to 0");

Review Comment:
   maybe: `Revoking assignments immediately since 
scheduled.rebalance.max.delay.ms is set to 0`



##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/IncrementalCooperativeAssignor.java:
##########
@@ -285,18 +292,13 @@ ClusterAssignment performTaskAssignment(
 
         handleLostAssignments(lostAssignments, newSubmissions, 
completeWorkerAssignment);
 
+        if (delay > 0) {
+            log.debug("Delaying {}ms for revoking tasks.", delay);
+        }
         // Do not revoke resources for re-assignment while a delayed rebalance 
is active
-        // Also we do not revoke in two consecutive rebalances by the same 
leader
-        canRevoke = delay == 0 && canRevoke;
-
-        // Compute the connectors-and-tasks to be revoked for load balancing 
without taking into
-        // account the deleted ones.
-        log.debug("Can leader revoke tasks in this assignment? {} (delay: 
{})", canRevoke, delay);
-        if (canRevoke) {
+        if (delay == 0) {
             Map<String, ConnectorsAndTasks> toExplicitlyRevoke =
-                    performTaskRevocation(activeAssignments, 
currentWorkerAssignment);
-
-            log.debug("Connector and task to revoke assignments: {}", 
toRevoke);
+                    performTaskRevocation(configured, 
completeWorkerAssignment);

Review Comment:
   @yashmayya , yes, your analysis is correct, I was trying to take the new 
added/deleted connectors/tasks into account for the revocation calculation, 
because I was trying to do consecutive revocation without delay.
   
   > But I'm just wondering if there's a reason why we were doing revocations 
only based on existing member assignments earlier.
   
   I guess that's because we didn't consider too complicated scenarios. Anyway, 
I think this change makes sense.



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to