lucasbru commented on code in PR #15511:
URL: https://github.com/apache/kafka/pull/15511#discussion_r1526346873


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java:
##########
@@ -911,9 +911,13 @@ void maybeReconcile() {
         SortedSet<TopicIdPartition> assignedTopicIdPartitions = 
findResolvableAssignmentAndTriggerMetadataUpdate();
         final LocalAssignmentImpl resolvedAssignment = new 
LocalAssignmentImpl(currentTargetAssignment.localEpoch, 
assignedTopicIdPartitions);
 
-        if (resolvedAssignment.equals(currentAssignment)) {
-            log.debug("Ignoring reconciliation attempt. Target assignment 
ready to reconcile {} " +
-                    "is equal to the member current assignment.", 
resolvedAssignment);
+        if (currentAssignment != LocalAssignmentImpl.NONE &&
+            resolvedAssignment.localEpoch <= currentAssignment.localEpoch + 1 
&&
+            
resolvedAssignment.partitions.equals(currentAssignment.partitions)) {
+            log.debug("Ignoring reconciliation attempt. The resolvable 
fragment of the target assignment {} " +

Review Comment:
   @dajac and I discussed various options to implement the short-cutting. In 
the end, the outcome was to only skip calling the callbacks but not skipping 
the entire reconciliation if the resolved assignment is equal to the current 
assignment but with a different epoch. We still need to update the current 
assignment with the resolved assignment and the new epoch in order to trigger 
the "ack", and transition to sending ack as well.



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