philipnee commented on code in PR #14690:
URL: https://github.com/apache/kafka/pull/14690#discussion_r1383658863


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/CommitRequestManager.java:
##########
@@ -166,19 +166,41 @@ private static long findMinTime(final Collection<? 
extends RequestState> request
             .orElse(Long.MAX_VALUE);
     }
 
-    public void maybeAutoCommit(final Map<TopicPartition, OffsetAndMetadata> 
offsets) {
+    /**
+     * Generate a request to commit offsets if auto-commit is enabled. The 
request will be
+     * returned to be sent out on the next call to {@link #poll(long)}. This 
will only generate a
+     * request if there is no other commit request already in-flight, and if 
the commit interval
+     * has elapsed.
+     *
+     * @param offsets Offsets to commit
+     * @return Future that will complete when a response is received for the 
request, or a
+     * completed future if no request is generated.
+     */
+    public CompletableFuture<Void> maybeAutoCommit(final Map<TopicPartition, 
OffsetAndMetadata> offsets) {
         if (!autoCommitState.isPresent()) {
-            return;
+            return CompletableFuture.completedFuture(null);
         }
 
         AutoCommitState autocommit = autoCommitState.get();
         if (!autocommit.canSendAutocommit()) {
-            return;
+            return CompletableFuture.completedFuture(null);
         }
 
-        sendAutoCommit(offsets);
+        CompletableFuture<Void> result = sendAutoCommit(offsets);
         autocommit.resetTimer();
         autocommit.setInflightCommitStatus(true);
+        return result;
+    }
+
+    /**
+     * If auto-commit is enabled, this will generate a commit offsets request 
for all assigned
+     * partitions and their current positions.
+     *
+     * @return Future that will complete when a response is received for the 
request, or a
+     * completed future if no request is generated.
+     */
+    public CompletableFuture<Void> maybeAutoCommitAllConsumed() {

Review Comment:
   Can we return an optional future here because there's technically nothing to 
be completed?



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

Reply via email to