FrankYang0529 commented on code in PR #17150:
URL: https://github.com/apache/kafka/pull/17150#discussion_r1795653405


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/ApplicationEventProcessor.java:
##########
@@ -173,22 +174,69 @@ private void process(final PollEvent event) {
 
     private void process(final AsyncCommitEvent event) {
         if (!requestManagers.commitRequestManager.isPresent()) {
+            event.future().complete(null);
             return;
         }
 
-        CommitRequestManager manager = 
requestManagers.commitRequestManager.get();
-        CompletableFuture<Void> future = manager.commitAsync(event.offsets());
-        future.whenComplete(complete(event.future()));
+        Map<TopicPartition, OffsetAndMetadata> offsets = 
event.offsets().isPresent() ?
+                event.offsets().get() : subscriptions.allConsumed();
+        if (offsets.isEmpty()) {
+            event.future().complete(Collections.emptyMap());
+            return;
+        }
+
+        try {
+            maybeUpdateLastSeenEpochIfNewer(offsets);
+            CommitRequestManager manager = 
requestManagers.commitRequestManager.get();
+            CompletableFuture<Void> future = manager.commitAsync(offsets);
+            future.whenComplete((r, e) -> {
+                if (e != null) {
+                    log.error("Committing offsets failed", e);
+                    event.future().completeExceptionally(e);
+                } else {
+                    event.future().complete(offsets);
+                }
+            });
+        } catch (Exception e) {
+            event.future().completeExceptionally(e);
+        }
     }
 
     private void process(final SyncCommitEvent event) {
         if (!requestManagers.commitRequestManager.isPresent()) {
+            event.future().complete(null);
             return;
         }
 
-        CommitRequestManager manager = 
requestManagers.commitRequestManager.get();
-        CompletableFuture<Void> future = manager.commitSync(event.offsets(), 
event.deadlineMs());
-        future.whenComplete(complete(event.future()));
+        Map<TopicPartition, OffsetAndMetadata> offsets = 
event.offsets().isPresent() ?
+                event.offsets().get() : subscriptions.allConsumed();
+        if (offsets.isEmpty()) {
+            event.future().complete(Collections.emptyMap());
+            return;
+        }
+
+        try {
+            maybeUpdateLastSeenEpochIfNewer(offsets);
+            CommitRequestManager manager = 
requestManagers.commitRequestManager.get();
+            CompletableFuture<Void> future = manager.commitSync(offsets, 
event.deadlineMs());
+            future.whenComplete((r, e) -> {
+                if (e != null) {
+                    log.error("Committing offsets failed", e);
+                    event.future().completeExceptionally(e);
+                } else {
+                    event.future().complete(offsets);
+                }
+            });
+        } catch (Exception e) {
+            event.future().completeExceptionally(e);
+        }
+    }
+
+    private void maybeUpdateLastSeenEpochIfNewer(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {
+        offsets.forEach((topicPartition, offsetAndMetadata) -> {
+            if (offsetAndMetadata.leaderEpoch().isPresent())
+                metadata.updateLastSeenEpochIfNewer(topicPartition, 
offsetAndMetadata.leaderEpoch().get());
+        });
     }

Review Comment:
   Agree, I extract a new function 
`ConsumerUtils#maybeUpdateLastSeenEpochIfNewer`. Thanks for the suggestion.



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

Reply via email to