RivenSun2 commented on a change in pull request #11340: URL: https://github.com/apache/kafka/pull/11340#discussion_r743682843
########## 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: sure ########## 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: 1. Yes. If (!autoCommitEnabled || coordinatorUnknown()), future will be null, I think in this case, we can give up committing offsets for rebalance 2&3. sure ########## 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: ok, I will add a comment here -- 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