lianetm commented on code in PR #15613:
URL: https://github.com/apache/kafka/pull/15613#discussion_r1543454136


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -1164,7 +1176,8 @@ public void maybeAutoCommitOffsetsAsync(long now) {
     }
 
     private boolean invokePendingAsyncCommits(Timer timer) {
-        if (inFlightAsyncCommits.get() == 0) {
+        if (pendingAsyncCommits.get() == 0 && inFlightAsyncCommits.get() == 0) 
{

Review Comment:
   This makes sense to me, to fill a gap in the case of commit sync with empty 
offsets, that skips the path of sending an actual request, and that's why it 
looses the guarantee of executing the callbacks as I see it.
   
   This makes the logic consistent with what happens if the commit sync has 
non-empty offsets. In that case, it does execute the callbacks for previous 
async commits that were waiting for coord:  the sync commit would be blocked on 
the same findCoord request (there's always just 1), and the moment the coord is 
found the async is marked as inflight 
[here](https://github.com/apache/kafka/blob/8b274d8c1bfbfa6d4319ded884a11da790d7bf77/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L1036),
 so it would be considered for callbacks 
[here](https://github.com/apache/kafka/blob/8b274d8c1bfbfa6d4319ded884a11da790d7bf77/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L1121).
 
   
   Am I getting the reasoning for the change right?



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