lianetm commented on code in PR #20363: URL: https://github.com/apache/kafka/pull/20363#discussion_r2288686114
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -859,16 +865,19 @@ public ConsumerRecords<K, V> poll(final Duration timeout) { throw new IllegalStateException("Consumer is not subscribed to any topics or assigned any partitions"); } - do { - PollEvent event = new PollEvent(timer.currentTimeMs()); - // Make sure to let the background thread know that we are still polling. - // This will trigger async auto-commits of consumed positions when hitting - // the interval time or reconciling new assignments - applicationEventHandler.add(event); + PollEvent event = new PollEvent(timer.currentTimeMs()); + // Make sure to let the background thread know that we are still polling. + // This will trigger async auto-commits of consumed positions when hitting + // the interval time or reconciling new assignments + applicationEventHandler.add(event); + + if (reconciliationInProgress.get() || autoCommitState.shouldAutoCommit()) { Review Comment: Couldn't we end up with a race condition here if the app thread sees `autoCommitState.shouldAutoCommit()` false at this point (because interval hasn't expired just yet), but by the time the background checks the same when processing the poll event the interval expired? In that case, I expect the background would trigger the auto-commit while the app thread moved onto updating positions for fetching (and that leads to a whole new set of race conditions that we already dealt with before). Basically, whatever change we introduce here to not wait on Poll, needs to ensure that we retrieve the positions to commit before moving on to update fetch positions, that's the main challenge with this change I expect. Thinking, but not sure yet how to address that if we don't wait on Poll. Thougts? -- 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