frankvicky commented on code in PR #18737: URL: https://github.com/apache/kafka/pull/18737#discussion_r1946904610
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java: ########## @@ -414,7 +415,15 @@ private void process(final ResetOffsetEvent event) { */ private void process(final CheckAndUpdatePositionsEvent event) { CompletableFuture<Boolean> future = requestManagers.offsetsRequestManager.updateFetchPositions(event.deadlineMs()); - future.whenComplete(complete(event.future())); + future.whenComplete((value, exception) -> { + if (exception != null) + event.future().completeExceptionally(exception); + else { + requestManagers.commitRequestManager.ifPresent(commitRequestManager -> + commitRequestManager.setLatestPartitionOffsets(subscriptions.allConsumed())); Review Comment: I'm considering adding a callback parameter to the `AsyncCommitEvent` constructor. However, this approach would require handling `InterruptedException` and `ExecutionException`, which isn't elegant. Alternatively, we could consider using nested events - just a rough idea I'm exploring. ```java private void commitAsync(Optional<Map<TopicPartition, OffsetAndMetadata>> offsets, OffsetCommitCallback callback) { acquireAndEnsureOpen(); try { CompletableFuture<Boolean> allConsumedIsDone = new CompletableFuture<>(); AsyncCommitEvent asyncCommitEvent = new AsyncCommitEvent(offsets, allConsumedIsDone); lastPendingAsyncCommit = commit(asyncCommitEvent).whenComplete((committedOffsets, throwable) -> { // omit unchaged part }); } finally { release(); } } private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commit(final CommitEvent commitEvent) { return this.commit(commitEvent, null); } private CompletableFuture<Map<TopicPartition, OffsetAndMetadata>> commit(final CommitEvent commitEvent, CompletableFuture<Void> allConsumeIsGet) { maybeThrowInvalidGroupIdException(); offsetCommitCallbackInvoker.executeCallbacks(); if (commitEvent.offsets().isPresent() && commitEvent.offsets().get().isEmpty()) { return CompletableFuture.completedFuture(null); } applicationEventHandler.add(commitEvent); if (allConsumeIsGet != null) { allConsumeIsGet.get(); } return commitEvent.future(); } ``` -- 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