kirktrue commented on code in PR #16310:
URL: https://github.com/apache/kafka/pull/16310#discussion_r1638835624
##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AsyncKafkaConsumer.java:
##########
@@ -1666,13 +1668,28 @@ private boolean initWithCommittedOffsetsIfNeeded(Timer
timer) {
return true;
log.debug("Refreshing committed offsets for partitions {}",
initializingPartitions);
+
+ // The shorter the timeout provided to poll(), the more likely the
offsets fetch will time out. To handle
+ // this case, on the first attempt to fetch the committed offsets, a
FetchCommittedOffsetsEvent is created
+ // (with potentially a longer timeout) and stored. The event is used
for the first attempt, but in the
+ // case it times out, subsequent attempts will also use the event in
order to wait for the results.
+ if (!canReusePendingOffsetFetchEvent(initializingPartitions)) {
+ // Give the event a reasonable amount of time to complete.
+ long timeoutMs = Math.max(defaultApiTimeoutMs,
timer.remainingMs());
+ long deadlineMs = time.milliseconds() + timeoutMs;
+ pendingOffsetFetchEvent = new
FetchCommittedOffsetsEvent(initializingPartitions, deadlineMs);
+ applicationEventHandler.add(pendingOffsetFetchEvent);
+ }
+
+ final CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> future
= pendingOffsetFetchEvent.future();
+
try {
- final FetchCommittedOffsetsEvent event =
- new FetchCommittedOffsetsEvent(
- initializingPartitions,
- calculateDeadlineMs(timer));
- wakeupTrigger.setActiveTask(event.future());
- final Map<TopicPartition, OffsetAndMetadata> offsets =
applicationEventHandler.addAndGet(event);
+ wakeupTrigger.setActiveTask(future);
+ final Map<TopicPartition, OffsetAndMetadata> offsets =
ConsumerUtils.getResult(future, timer);
Review Comment:
Is there any _harm_ in having `InterruptException` clear out the pending
event? Why else would an interrupt exception be thrown in normal operations?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]