dajac commented on code in PR #14557: URL: https://github.com/apache/kafka/pull/14557#discussion_r1426720259
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java: ########## @@ -1119,8 +1125,11 @@ public void commitSync(Map<TopicPartition, OffsetAndMetadata> offsets, Duration acquireAndEnsureOpen(); long commitStart = time.nanoseconds(); try { - CompletableFuture<Void> commitFuture = commit(offsets, true); - ConsumerUtils.getResult(commitFuture, time.timer(timeout)); + Timer requestTimer = time.timer(timeout.toMillis()); + // Commit with a timer to control how long the request should be retried until it + // gets a successful response or non-retriable error. + CompletableFuture<Void> commitFuture = commit(offsets, true, Optional.of(timeout.toMillis())); + ConsumerUtils.getResult(commitFuture, requestTimer); Review Comment: Let me see if I understand this correctly. It seems to me that we have two timers on this path. A first one here which bound the time waiting on the result. A second one in the background thread to no retry forever. Is my understanding correct? If my understanding is correct, there is a chance that the first timer may timeout before the background thread had a change to complete the future. In this case, could we lose the real background exception? -- 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