@ -161,7 +161,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -161,7 +161,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -192,7 +192,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -192,7 +192,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -864,7 +864,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -864,7 +864,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
finalintminUserMetadataVersion,
finalintminSupportedMetadataVersion,
finalbooleanshouldTriggerProbingRebalance){
// keep track of whether a 2nd rebalance is unavoidable so we can skip trying to get a completely sticky assignment
@ -878,8 +877,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -878,8 +877,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
// Try to avoid triggering another rebalance by giving active tasks back to their previous owners within a
// client, without violating load balance. If we already know another rebalance will be required, or the
// client had no owned partitions, try to balance the workload as evenly as possible by interleaving the
// tasks among consumers and hopefully spreading the heavier subtopologies evenly across threads.
// client had no owned partitions, try to balance the workload as evenly as possible by interleaving tasks
@ -894,7 +892,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -894,7 +892,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
// Arbitrarily choose the leader's client to be responsible for triggering the probing rebalance
@ -907,6 +905,17 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -907,6 +905,17 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
minSupportedMetadataVersion,
false,
encodeNextProbingRebalanceTime);
if(followupRebalanceScheduled){
rebalanceRequired=true;
log.debug("Requested client {} to schedule a followup rebalance",clientId);
}
}
if(rebalanceRequired){
log.info("Finished unstable assignment of tasks, a followup rebalance will be scheduled.");
}else{
log.info("Finished stable assignment of tasks, no followup rebalances required.");
}
returnassignment;
@ -955,26 +964,29 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -955,26 +964,29 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
false);
}
log.info("Finished unstable assignment of tasks, a followup rebalance will be scheduled due to version probing.");
@ -984,17 +996,14 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -984,17 +996,14 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1009,14 +1018,18 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1009,14 +1018,18 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
AssignorError.NONE.code()
);
if(encodeNextRebalanceTime){
if(tasksRevoked){
// TODO: once KAFKA-9821 is resolved we can leave it to the client to trigger this rebalance
log.debug("Requesting followup rebalance be scheduled immediately due to tasks changing ownership.");
log.debug("Requesting followup rebalance be scheduled for {} ms to probe for caught-up replica tasks.",nextRebalanceTimeMs);
info.setNextRebalanceTime(nextRebalanceTimeMs);
log.info("Scheduled a followup probing rebalance for {} ms.",nextRebalanceTimeMs);
encodeNextRebalanceTime=false;
shouldEncodeProbingRebalance=false;
}
// finally, encode the assignment and insert into map with all assignments
assignment.put(
consumer,
newAssignment(
@ -1025,12 +1038,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1025,12 +1038,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
)
);
}
if(stableAssignment){
log.info("Finished stable assignment of tasks, no followup rebalances required.");
}else{
log.info("Finished unstable assignment of tasks, a followup probing rebalance will be triggered.");
}
returnrebalanceRequested;
}
/**
@ -1057,7 +1065,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1057,7 +1065,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
log.info("Removing task {} from assignment until it is safely revoked in followup rebalance",taskId);
clientState.removeFromAssignment(taskId);
@ -1338,7 +1346,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1338,7 +1346,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1356,18 +1364,13 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1356,18 +1364,13 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1377,6 +1380,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1377,6 +1380,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
partitionsByHost=Collections.emptyMap();
standbyPartitionsByHost=Collections.emptyMap();
topicToPartitionInfo=Collections.emptyMap();
encodedNextScheduledRebalanceMs=Long.MAX_VALUE;
break;
case2:
case3:
@ -1388,6 +1392,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1388,6 +1392,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1396,6 +1401,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1396,6 +1401,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1404,7 +1410,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1404,7 +1410,7 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1413,7 +1419,12 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1413,7 +1419,12 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1423,6 +1434,28 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1423,6 +1434,28 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1430,15 +1463,15 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1430,15 +1463,15 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -1543,10 +1576,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@@ -1543,10 +1576,6 @@ public class StreamsPartitionAssignor implements ConsumerPartitionAssignor, Conf
@ -31,23 +32,26 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
@@ -31,23 +32,26 @@ public class StreamsRebalanceListener implements ConsumerRebalanceListener {
@ -123,7 +123,7 @@ public class HighAvailabilityStreamsPartitionAssignorTest {
@@ -123,7 +123,7 @@ public class HighAvailabilityStreamsPartitionAssignorTest {
@ -528,22 +528,22 @@ class StreamsUpgradeTest(Test):
@@ -528,22 +528,22 @@ class StreamsUpgradeTest(Test):
log_monitor.wait_until("Sent a version 8 subscription and got version 7 assignment back (successful version probing). Downgrade subscription metadata to commonly supported version 7 and trigger new rebalance.",
timeout_sec=60,
err_msg="Could not detect 'successful version probing' at upgrading node "+str(node.account))
log_monitor.wait_until("Detected that the assignor requested a rebalance. Rejoining the consumer group to trigger a new rebalance.",
log_monitor.wait_until("Triggering the followup rebalance scheduled for 0 ms.",
timeout_sec=60,
err_msg="Could not detect 'Triggering new rebalance' at upgrading node "+str(node.account))
err_msg="Could not detect 'Triggering followup rebalance' at upgrading node "+str(node.account))
else:
first_other_monitor.wait_until("Sent a version 7 subscription and group.s latest commonly supported version is 8 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 8 for next rebalance.",
timeout_sec=60,
err_msg="Never saw output 'Upgrade metadata to version 8' on"+str(first_other_node.account))
first_other_monitor.wait_until("Detected that the assignor requested a rebalance. Rejoining the consumer group to trigger a new rebalance.",
first_other_monitor.wait_until("Triggering the followup rebalance scheduled for 0 ms.",
timeout_sec=60,
err_msg="Could not detect 'Triggering new rebalance' at upgrading node "+str(node.account))
err_msg="Could not detect 'Triggering followup rebalance' at upgrading node "+str(node.account))
second_other_monitor.wait_until("Sent a version 7 subscription and group.s latest commonly supported version is 8 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 8 for next rebalance.",
timeout_sec=60,
err_msg="Never saw output 'Upgrade metadata to version 8' on"+str(second_other_node.account))
second_other_monitor.wait_until("Detected that the assignor requested a rebalance. Rejoining the consumer group to trigger a new rebalance.",
second_other_monitor.wait_until("Triggering the followup rebalance scheduled for 0 ms.",
timeout_sec=60,
err_msg="Could not detect 'Triggering new rebalance' at upgrading node "+str(node.account))
err_msg="Could not detect 'Triggering followup rebalance' at upgrading node "+str(node.account))
# version probing should trigger second rebalance
# now we check that after consecutive rebalances we have synchronized generation