[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-12-28 Thread GitBox


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



##
File path: 
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
##
@@ -3202,17 +3202,18 @@ private void gracefulCloseTest(ConsumerCoordinator 
coordinator, boolean shouldLe
 OffsetCommitRequest commitRequest = (OffsetCommitRequest) body;
 return commitRequest.data().groupId().equals(groupId);
 }, new OffsetCommitResponse(new OffsetCommitResponseData()));
+if (shouldLeaveGroup)

Review comment:
   Could you explain why we need to check `shouldLeaveGroup` after your 
change?

##
File path: 
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##
@@ -2230,7 +2230,7 @@ public void testInvalidGroupMetadata() throws 
InterruptedException {
 MockClient client = new MockClient(time, metadata);
 initMetadata(client, Collections.singletonMap(topic, 1));
 KafkaConsumer consumer = newConsumer(time, client, 
subscription, metadata,
-new RoundRobinAssignor(), true, groupInstanceId);
+new RoundRobinAssignor(), false, groupInstanceId);

Review comment:
   I'm afraid the change of `autoCommit` flag will change the test purpose. 
I checked and it looks like we'll wait for consumer to close forever, because 
the timer didn't tick. Could we do `consumer.close(Duration.ZERO);`, and keep 
the `autoCommit` as true as before? What do you think?




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




[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-12-28 Thread GitBox


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



##
File path: 
clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##
@@ -2230,7 +2230,7 @@ public void testInvalidGroupMetadata() throws 
InterruptedException {
 MockClient client = new MockClient(time, metadata);
 initMetadata(client, Collections.singletonMap(topic, 1));
 KafkaConsumer consumer = newConsumer(time, client, 
subscription, metadata,
-new RoundRobinAssignor(), true, groupInstanceId);
+new RoundRobinAssignor(), false, groupInstanceId);

Review comment:
   I understand the mockTime change below. But why should we change the 
`autoCommit` flag 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




[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -692,10 +692,25 @@ private void validateCooperativeAssignment(final 
Map // null future means no offset commit request sent, so it is still 
considered completed

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -692,10 +692,25 @@ private void validateCooperativeAssignment(final 
Map future = maybeAutoCommitOffsetsAsync();
+if (future == null)
+onJoinPrepareAsyncCommitCompleted = true;
+else {
+if (future.succeeded()) {
+onJoinPrepareAsyncCommitCompleted = true;
+} else if (future.failed()) {
+// consistent with async auto-commit failures, we do not 
propagate the exception
+log.warn("Asynchronous auto-commit offsets failed: {}", 
future.exception().getMessage());

Review comment:
   Sorry, I found we already log the error in `autoCommitOffsetsAsync`. We 
should remove the logging here. And make the if condition simpler as: 
   ```
   if (future == null || future.succeeded() || (future.failed() && 
!future.isRetriable())) {
   onJoinPrepareAsyncCommitCompleted = true;
   }

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -692,10 +692,25 @@ private void validateCooperativeAssignment(final 
Map future = maybeAutoCommitOffsetsAsync();
+if (future == null)
+onJoinPrepareAsyncCommitCompleted = true;
+else {
+if (future.succeeded()) {
+onJoinPrepareAsyncCommitCompleted = true;
+} else if (future.failed()) {
+// consistent with async auto-commit failures, we do not 
propagate the exception
+log.warn("Asynchronous auto-commit offsets failed: {}", 
future.exception().getMessage());

Review comment:
   Actually, after the concise if statement above, we can add comments like 
this:
   ```
   // return true when
   // 1. future is null, which means no commit request sent
   // 2. offset commit completed
   // 3. offset commit failed with non-retriable error
   ```

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1073,22 +1092,14 @@ private void doAutoCommitOffsetsAsync() {
 });
 }
 
-private void maybeAutoCommitOffsetsSync(Timer timer) {
+private RequestFuture maybeAutoCommitOffsetsAsync() {
 if (autoCommitEnabled) {
-Map allConsumedOffsets = 
subscriptions.allConsumed();
-try {
-log.debug("Sending synchronous auto-commit of offsets {}", 
allConsumedOffsets);
-if (!commitOffsetsSync(allConsumedOffsets, timer))
-log.debug("Auto-commit of offsets {} timed out before 
completion", allConsumedOffsets);
-} catch (WakeupException | InterruptException e) {
-log.debug("Auto-commit of offsets {} was interrupted before 
completion", allConsumedOffsets);
-// rethrow wakeups since they are triggered by the user
-throw e;
-} catch (Exception e) {
-// consistent with async auto-commit failures, we do not 
propagate the exception
-log.warn("Synchronous auto-commit of offsets {} failed: {}", 
allConsumedOffsets, e.getMessage());
-}
-}
+RequestFuture future = autoCommitOffsetsAsync();
+client.pollNoWakeup();
+invokeCompletedOffsetCommitCallbacks();
+return future;
+} else
+return null;

Review comment:
   nit: remove `else` here, return null at the end.




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




[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-05 Thread GitBox


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

[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-05 Thread GitBox


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

[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-05 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -692,10 +693,18 @@ private void validateCooperativeAssignment(final 
Map

[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-05 Thread GitBox


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

[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-03 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##
@@ -420,9 +421,14 @@ 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))

Review comment:
   You're right. How about this:
   ```java
   
   if (needsJoinPrepare) {
   try {
   if (onJoinPrepare(generation.generationId, generation.memberId))
   needsJoinPrepare = false;
   else
   return false;
   } catch (KafkaException e) {
   // if onJoinPrepare throws an exception, it would be from the 
rebalance listener.
   // next time we would then not retry {@code onJoinPrepare} any 
more but proceed the join-group procedure.
   needsJoinPrepare = false;
   throw e;
   }
   }
   ```




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




[GitHub] [kafka] showuon commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-11-03 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -692,10 +693,18 @@ private void validateCooperativeAssignment(final 
Map() {

Review comment:
   Since this variable only used in the new added test, could we put it 
into the test?

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1073,6 +1086,33 @@ private void doAutoCommitOffsetsAsync() {
 });
 }
 
+private boolean maybeAutoCommitOffsetsAsync() {
+if (autoCommitEnabled) {
+invokeCompletedOffsetCommitCallbacks();
+
+if (onJoinPrepareAsyncCommitFuture == null)
+onJoinPrepareAsyncCommitFuture = doAutoCommitOffsetsAsync();
+if (onJoinPrepareAsyncCommitFuture == null)
+return true;
+
+client.pollNoWakeup();
+invokeCompletedOffsetCommitCallbacks();
+
+if (!onJoinPrepareAsyncCommitFuture.isDone())
+return false;
+if (onJoinPrepareAsyncCommitFuture.succeeded()) {
+onJoinPrepareAsyncCommitFuture = null;
+return true;
+}
+if (onJoinPrepareAsyncCommitFuture.failed() && 
!onJoinPrepareAsyncCommitFuture.isRetriable())

Review comment:
   +1




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