showuon commented on a change in pull request #11340:
URL: https://github.com/apache/kafka/pull/11340#discussion_r743635124



##########
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -692,10 +692,22 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected void onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(int generation, String memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        // commit offsets prior to rebalance if auto-commit enabled
-        
maybeAutoCommitOffsetsSync(time.timer(rebalanceConfig.rebalanceTimeoutMs));
+        boolean onJoinPrepareAsyncCommitSucceeded = false;
+        // async commit offsets prior to rebalance if auto-commit enabled
+        RequestFuture<Void> future = maybeAutoCommitOffsetsAsync();
+        if (future == null)
+            onJoinPrepareAsyncCommitSucceeded = true;
+        else {
+            if (future.succeeded()) {
+                onJoinPrepareAsyncCommitSucceeded = true;
+            } else if (future.failed() && !future.isRetriable()) {
+                // consistent with async auto-commit failures, we do not 
propagate the exception
+                log.warn("Asynchronous auto-commit offsets failed: {}", 
future.exception().getMessage());
+                onJoinPrepareAsyncCommitSucceeded = true;
+            }

Review comment:
       I think we should also log the error in `failed && isRetriable()` case

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -692,10 +693,18 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected void onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(int generation, String memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        // commit offsets prior to rebalance if auto-commit enabled
-        
maybeAutoCommitOffsetsSync(time.timer(rebalanceConfig.rebalanceTimeoutMs));
+        boolean onJoinPrepareAsyncCommitSucceeded;
+        try {
+            // async commit offsets prior to rebalance if auto-commit enabled
+            onJoinPrepareAsyncCommitSucceeded = maybeAutoCommitOffsetsAsync();
+        } catch (Exception e) {
+            onJoinPrepareAsyncCommitFuture = null;
+            onJoinPrepareAsyncCommitSucceeded = true;

Review comment:
       So, 2 comments here:
   1. If we return `true`, we won't do `onJoinPrepare` on next poll, and the 
offset commit will be skipped. Is that expected? 
   2. the returned `future` null means no commit operated (since no auto 
commit). Could you add a comment here to explain the case?
   3. The variable name `onJoinPrepareAsyncCommitSucceeded` might be better to 
change to `onJoinPrepareAsyncCommitCompleted` ? WDYT?
   

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -420,7 +421,11 @@ boolean joinGroupIfNeeded(final Timer timer) {
                 // need to set the flag before calling onJoinPrepare since the 
user callback may throw
                 // exception, in which case upon retry we should not retry 
onJoinPrepare either.
                 needsJoinPrepare = false;
-                onJoinPrepare(generation.generationId, generation.memberId);
+                if (!onJoinPrepare(generation.generationId, 
generation.memberId)) {
+                    needsJoinPrepare = true;
+                    //should not initiateJoinGroup if needsJoinPrepare still 
is true

Review comment:
       nit: // return false when onJoinPrepare is waiting for committing offset

##########
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -692,10 +693,18 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected void onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(int generation, String memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        // commit offsets prior to rebalance if auto-commit enabled
-        
maybeAutoCommitOffsetsSync(time.timer(rebalanceConfig.rebalanceTimeoutMs));
+        boolean onJoinPrepareAsyncCommitSucceeded;
+        try {
+            // async commit offsets prior to rebalance if auto-commit enabled
+            onJoinPrepareAsyncCommitSucceeded = maybeAutoCommitOffsetsAsync();
+        } catch (Exception e) {
+            onJoinPrepareAsyncCommitFuture = null;
+            onJoinPrepareAsyncCommitSucceeded = true;

Review comment:
       So, 3 comments here:
   1. If we return `true`, we won't do `onJoinPrepare` on next poll, and the 
offset commit will be skipped. Is that expected? 
   2. the returned `future` null means no commit operated (since no auto 
commit). Could you add a comment here to explain the case?
   3. The variable name `onJoinPrepareAsyncCommitSucceeded` might be better to 
change to `onJoinPrepareAsyncCommitCompleted` ? WDYT?
   




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