showuon commented on code in PR #12349:
URL: https://github.com/apache/kafka/pull/12349#discussion_r921765863


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -740,24 +743,45 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected boolean onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(Timer timer, int generation, String 
memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        boolean onJoinPrepareAsyncCommitCompleted = false;
+        if (joinPrepareTimer == null) {
+            joinPrepareTimer = time.timer(rebalanceConfig.rebalanceTimeoutMs);
+        }
         // async commit offsets prior to rebalance if auto-commit enabled
-        RequestFuture<Void> future = maybeAutoCommitOffsetsAsync();
-        // return true when
-        // 1. future is null, which means no commit request sent, so it is 
still considered completed
-        // 2. offset commit completed
-        // 3. offset commit failed with non-retriable exception
-        if (future == null)
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.succeeded())
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.failed() && !future.isRetriable()) {
-            log.error("Asynchronous auto-commit of offsets failed: {}", 
future.exception().getMessage());
-            onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+            autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
+        }
+
+        // wait for commit offset response before timer.
+        if (autoCommitOffsetRequestFuture != null) {
+            Timer pollTimer = timer.remainingMs() < 
joinPrepareTimer.remainingMs() ?
+                   timer : joinPrepareTimer;
+            client.poll(autoCommitOffsetRequestFuture, pollTimer);
         }
 
+        // return false when:
+        //   1. offset commit haven't done
+        //   2. offset commit failed with retriable exception and joinPrepare 
haven't expired
+        boolean onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitOffsetRequestFuture != null) {
+            if (!autoCommitOffsetRequestFuture.isDone()) {
+                onJoinPrepareAsyncCommitCompleted = false;
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                onJoinPrepareAsyncCommitCompleted = 
joinPrepareTimer.isExpired();
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                log.error("Asynchronous auto-commit of offsets failed: {}", 
autoCommitOffsetRequestFuture.exception().getMessage());

Review Comment:
   These 2 else if conditions are the same. Please fix it.



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -740,24 +743,45 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected boolean onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(Timer timer, int generation, String 
memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        boolean onJoinPrepareAsyncCommitCompleted = false;
+        if (joinPrepareTimer == null) {
+            joinPrepareTimer = time.timer(rebalanceConfig.rebalanceTimeoutMs);
+        }
         // async commit offsets prior to rebalance if auto-commit enabled
-        RequestFuture<Void> future = maybeAutoCommitOffsetsAsync();
-        // return true when
-        // 1. future is null, which means no commit request sent, so it is 
still considered completed
-        // 2. offset commit completed
-        // 3. offset commit failed with non-retriable exception
-        if (future == null)
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.succeeded())
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.failed() && !future.isRetriable()) {
-            log.error("Asynchronous auto-commit of offsets failed: {}", 
future.exception().getMessage());
-            onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+            autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
+        }
+
+        // wait for commit offset response before timer.
+        if (autoCommitOffsetRequestFuture != null) {
+            Timer pollTimer = timer.remainingMs() < 
joinPrepareTimer.remainingMs() ?
+                   timer : joinPrepareTimer;
+            client.poll(autoCommitOffsetRequestFuture, pollTimer);
         }
 
+        // return false when:
+        //   1. offset commit haven't done
+        //   2. offset commit failed with retriable exception and joinPrepare 
haven't expired
+        boolean onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitOffsetRequestFuture != null) {
+            if (!autoCommitOffsetRequestFuture.isDone()) {
+                onJoinPrepareAsyncCommitCompleted = false;

Review Comment:
   I think we should also check `joinPrepareTimer.isExpired();` here, right?



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -740,24 +743,45 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected boolean onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(Timer timer, int generation, String 
memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        boolean onJoinPrepareAsyncCommitCompleted = false;
+        if (joinPrepareTimer == null) {
+            joinPrepareTimer = time.timer(rebalanceConfig.rebalanceTimeoutMs);
+        }
         // async commit offsets prior to rebalance if auto-commit enabled
-        RequestFuture<Void> future = maybeAutoCommitOffsetsAsync();
-        // return true when
-        // 1. future is null, which means no commit request sent, so it is 
still considered completed
-        // 2. offset commit completed
-        // 3. offset commit failed with non-retriable exception
-        if (future == null)
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.succeeded())
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.failed() && !future.isRetriable()) {
-            log.error("Asynchronous auto-commit of offsets failed: {}", 
future.exception().getMessage());
-            onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+            autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
+        }
+
+        // wait for commit offset response before timer.
+        if (autoCommitOffsetRequestFuture != null) {
+            Timer pollTimer = timer.remainingMs() < 
joinPrepareTimer.remainingMs() ?
+                   timer : joinPrepareTimer;
+            client.poll(autoCommitOffsetRequestFuture, pollTimer);
         }
 
+        // return false when:
+        //   1. offset commit haven't done
+        //   2. offset commit failed with retriable exception and joinPrepare 
haven't expired
+        boolean onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitOffsetRequestFuture != null) {
+            if (!autoCommitOffsetRequestFuture.isDone()) {
+                onJoinPrepareAsyncCommitCompleted = false;
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                onJoinPrepareAsyncCommitCompleted = 
joinPrepareTimer.isExpired();
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                log.error("Asynchronous auto-commit of offsets failed: {}", 
autoCommitOffsetRequestFuture.exception().getMessage());
+            } else if (joinPrepareTimer != null && 
joinPrepareTimer.isExpired()) {
+                log.error("Asynchronous auto-commit of offsets failed: 
joinPrepare timeout");
+            }
+            if (autoCommitOffsetRequestFuture.isDone()) {
+                autoCommitOffsetRequestFuture = null;
+            }
+        }
+        if (!onJoinPrepareAsyncCommitCompleted) {
+            timer.sleep(rebalanceConfig.retryBackoffMs);

Review Comment:
   I don't think we should backoff here, or we should backoff 
`Math.min(pollTimer.remainingMs, rebalanceConfig.retryBackoffMs)`



##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java:
##########
@@ -740,24 +743,45 @@ private void validateCooperativeAssignment(final 
Map<String, List<TopicPartition
     }
 
     @Override
-    protected boolean onJoinPrepare(int generation, String memberId) {
+    protected boolean onJoinPrepare(Timer timer, int generation, String 
memberId) {
         log.debug("Executing onJoinPrepare with generation {} and memberId 
{}", generation, memberId);
-        boolean onJoinPrepareAsyncCommitCompleted = false;
+        if (joinPrepareTimer == null) {
+            joinPrepareTimer = time.timer(rebalanceConfig.rebalanceTimeoutMs);
+        }
         // async commit offsets prior to rebalance if auto-commit enabled
-        RequestFuture<Void> future = maybeAutoCommitOffsetsAsync();
-        // return true when
-        // 1. future is null, which means no commit request sent, so it is 
still considered completed
-        // 2. offset commit completed
-        // 3. offset commit failed with non-retriable exception
-        if (future == null)
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.succeeded())
-            onJoinPrepareAsyncCommitCompleted = true;
-        else if (future.failed() && !future.isRetriable()) {
-            log.error("Asynchronous auto-commit of offsets failed: {}", 
future.exception().getMessage());
-            onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitEnabled && autoCommitOffsetRequestFuture == null) {
+            autoCommitOffsetRequestFuture = maybeAutoCommitOffsetsAsync();
+        }
+
+        // wait for commit offset response before timer.
+        if (autoCommitOffsetRequestFuture != null) {
+            Timer pollTimer = timer.remainingMs() < 
joinPrepareTimer.remainingMs() ?
+                   timer : joinPrepareTimer;
+            client.poll(autoCommitOffsetRequestFuture, pollTimer);
         }
 
+        // return false when:
+        //   1. offset commit haven't done
+        //   2. offset commit failed with retriable exception and joinPrepare 
haven't expired
+        boolean onJoinPrepareAsyncCommitCompleted = true;
+        if (autoCommitOffsetRequestFuture != null) {
+            if (!autoCommitOffsetRequestFuture.isDone()) {
+                onJoinPrepareAsyncCommitCompleted = false;
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                onJoinPrepareAsyncCommitCompleted = 
joinPrepareTimer.isExpired();
+            } else if (autoCommitOffsetRequestFuture.failed() && 
autoCommitOffsetRequestFuture.isRetriable()) {
+                log.error("Asynchronous auto-commit of offsets failed: {}", 
autoCommitOffsetRequestFuture.exception().getMessage());
+            } else if (joinPrepareTimer != null && 
joinPrepareTimer.isExpired()) {

Review Comment:
   Curious: should we need null check here? Will this `onJoinPrepare` methodh 
be called by other thread?



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