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


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -324,7 +323,7 @@ public CompletableFuture<Void> 
maybeAutoCommitSyncBeforeRevocation(final long de
 
         CompletableFuture<Void> result = new CompletableFuture<>();
         OffsetCommitRequestState requestState =
-            createOffsetCommitRequest(subscriptions.allConsumed(), deadlineMs);
+            createOffsetCommitRequest(latestPartitionOffsets, deadlineMs);

Review Comment:
   yes, I get your concern (and I'm still going over all this), but this is how 
I see it now:
   
   1. we would be only waiting to trigger the reconciliation, not to process it 
until the end (once triggered, it will carry on in the background as always, 
not blocking on anything new, just the commit request to complete and the 
callbacks, as always)
   3. we're just saying we will align the reconciliation triggering with the 
consumer poll (like the classic does btw) because we need to wait for stable 
positions to start reconciling a new assignment. So yes, there is a delay to 
start reconciling, but it's for correctness: we have to commit before a 
rebalance, but we cannot guarantee we can commit correctly the consumed 
positions if we don't have stable positions.
   
   Looking at the poll loop from a high level, these are the 3 main blocks:
    
   1. app thread poll start (PollEvent) 
   2. update fetch positions
   3. fetch
   
   So we're saying we trigger a reconciliation only on 1, when we have stable 
positions, so we know the allConsumed to commit (and then rebalance). Of course 
that means that if we get a HB response with a new assignment right after the 
`PollEvent` (1), we would have to wait until the next `PollEvent` to start 
reconciling that assignment. But with the current version of triggering 
reconciliations freely in the background, that's exactly the root cause of the 
problem imo: we start a reconciliation when 2/3 are happening, and it's a mess 
because we cannot determine the `allConsumed` to commit, it's a moving target 
(until we know the records have been returned, and that's on the next 
PollEvent).
   
   Thoughts?



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