[GitHub] [kafka] RivenSun2 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


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



##
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:
   1.Sure, `consumer.close(Duration.ZERO)` is also a solution. Thanks.
   My previous changes should not change the purpose of this test, and will not 
cause us to wait for the consumer to close forever. Because `autoCommit` is 
disabled, the `ConsumerCoordinator#close` method will end soon.
   Local verification is also performed, `testInvalidGroupMetadata` case can be 
passed quickly and successfully
   
   2.The `gracefulCloseTest` method is only called by two testCases, 
`testCloseDynamicAssignment` and `testCloseManualAssignment`.
   `testCloseDynamicAssignment` uses the subscribe consumption model,
   `testCloseManualAssignment` uses the assign consumption model. The 
difference here is: when `coordinator.close()` is executed, 
`testCloseDynamicAssignment` will initiate a LeaveGroupRequest in the 
underlying `AbstractCoordinator#maybeLeaveGroup(String leaveReason)` method, 
but `testCloseManualAssignment` will not.
   So when I prepared `LeaveGroupResponse` here, I added a judgment on 
`shouldLeaveGroup`.
   The previous `gracefulCloseTest` code logic should prepare redundant 
`LeaveGroupResponse` for `testCloseManualAssignment`.




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


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



##
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:
   1.Sure, `consumer.close(Duration.ZERO)` is also a solution. Thanks.
   My previous changes should not change the purpose of this test, and will not 
cause us to wait for the consumer to close forever. Because `autoCommit` is 
disabled, the `ConsumerCoordinator#close` method will end soon.
   
   2.The `gracefulCloseTest` method is only called by two testCases, 
`testCloseDynamicAssignment` and `testCloseManualAssignment`.
   `testCloseDynamicAssignment` uses the subscribe consumption model,
   `testCloseManualAssignment` uses the assign consumption model. The 
difference here is: when `coordinator.close()` is executed, 
`testCloseDynamicAssignment` will initiate a LeaveGroupRequest in the 
underlying `AbstractCoordinator#maybeLeaveGroup(String leaveReason)` method, 
but `testCloseManualAssignment` will not.
   So when I prepared `LeaveGroupResponse` here, I added a judgment on 
`shouldLeaveGroup`.
   The previous `gracefulCloseTest` code logic should prepare redundant 
`LeaveGroupResponse` for `testCloseManualAssignment`.




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


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



##
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:
   1.Sure, `consumer.close(Duration.ZERO)` is also a solution, thanks.
   
   2.The `gracefulCloseTest` method is only called by two testCases, 
`testCloseDynamicAssignment` and `testCloseManualAssignment`.
   `testCloseDynamicAssignment` uses the subscribe consumption model,
   `testCloseManualAssignment` uses the assign consumption model. The 
difference here is: when `coordinator.close()` is executed, 
`testCloseDynamicAssignment` will initiate a LeaveGroupRequest in the 
underlying `AbstractCoordinator#maybeLeaveGroup(String leaveReason)` method, 
but `testCloseManualAssignment` will not.
   So when I prepared `LeaveGroupResponse` here, I added a judgment on 
`shouldLeaveGroup`.
   The previous `gracefulCloseTest` code logic should prepare redundant 
`LeaveGroupResponse` for `testCloseManualAssignment`.




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


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



##
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:
   Since the `ConsumerCoordinator#close` method will call 
`maybeAutoCommitOffsetsAsync()`, we need to add an additional OffsetCommit 
prepareResponse in `testCloseManualAssignment` case.
   The code has been submitted, please help check it, thanks @showuon 




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


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



##
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:
   Because this testInvalidGroupMetadata deliberately calls `new 
MockTime()` to make `autoTickMs=0`, I cannot set autoTickMs to non-zero here.
   `() -> consumer.poll(Duration.ofSeconds(5))` this line of code will block, 
so that 
`assertThrows(ConcurrentModificationException.class,consumer::groupMetadata)` 
can pass,
   Then call `consumer.wakeup()` to make `() -> 
consumer.poll(Duration.ofSeconds(5))` throw `WakeupException` and exit blocking.
   `assertTrue(service.awaitTermination(10, TimeUnit.SECONDS))` can also be 
passed




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-12-22 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1073,22 +1084,14 @@ private void doAutoCommitOffsetsAsync() {
 });
 }
 
-private void maybeAutoCommitOffsetsSync(Timer timer) {
+private RequestFuture maybeAutoCommitOffsetsAsync() {

Review comment:
   I agree with you,thanks




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


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

[GitHub] [kafka] RivenSun2 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


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

[GitHub] [kafka] RivenSun2 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


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



##
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] RivenSun2 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


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




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


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1073,6 +1086,33 @@ private void doAutoCommitOffsetsAsync() {
 });
 }
 
+private boolean maybeAutoCommitOffsetsAsync() {

Review comment:
   In fact, only onJoinPrepare called the maybeAutoCommitOffsetsAsync 
method before.




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


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



##
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:
   I think it can be simplified like this
```
  if (needsJoinPrepare) {
   // 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;
   if (!onJoinPrepare(generation.generationId, 
generation.memberId)) {
   needsJoinPrepare = true;
   //should not initiateJoinGroup if needsJoinPrepare still 
is true
   return false;
   }
   }
   ```




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


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



##
File path: 
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
##
@@ -1053,6 +1058,49 @@ public void 
testForceMetadataRefreshForPatternSubscriptionDuringRebalance() {
 assertFalse(coordinator.rejoinNeededOrPending());
 }
 
+@Test
+public void testForceMetadataDeleteForPatternSubscriptionDuringRebalance() 
{

Review comment:
   In fact, only onJoinPrepare called the maybeAutoCommitOffsetsAsync 
method before.
   Follow guozhang's suggestion , I will delete the 
maybeAutoCommitOffsetsSync(timer) method,
   close() also calls maybeAutoCommitOffsetsAsync




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


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



##
File path: 
clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
##
@@ -151,6 +151,11 @@
 put(topic2, 1);
 }
 });
+private MetadataResponse deletedMetadataResponse = 
RequestTestUtils.metadataUpdateWith(1, new HashMap() {

Review comment:
   Sure
   




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


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



##
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] RivenSun2 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


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##
@@ -403,7 +404,7 @@ private void closeHeartbeatThread() {
  *
  * @param timer Timer bounding how long this method can block
  * @throws KafkaException if the callback throws exception
- * @return true iff the operation succeeded
+ * @return true if the operation succeeded

Review comment:
   my fault, I thought it was a misspelling of a word, I will restore it




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


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



##
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();

Review comment:
   my fault, already fix it




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


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



##
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:
   If you do
   This goal should not be achieved:
   ```
   if (!onJoinPrepare(generation.generationId, generation.memberId))
needsJoinPrepare = true;
   ```




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


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



##
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:
   If you do
   This goal should not be achieved:
   ```
   if (!onJoinPrepare(generation.generationId, generation.memberId))
needsJoinPrepare = true;
   ```




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


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



##
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:
   I agree with your suggestion.




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


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1073,6 +1086,33 @@ private void doAutoCommitOffsetsAsync() {
 });
 }
 
+private boolean maybeAutoCommitOffsetsAsync() {

Review comment:
   I agree with your suggestion.

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -1054,11 +1067,11 @@ public void maybeAutoCommitOffsetsAsync(long now) {
 }
 }
 
-private void doAutoCommitOffsetsAsync() {

Review comment:
   I agree with your suggestion.




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


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



##
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] RivenSun2 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


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



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

Review comment:
   Because we use asynchronous commitOffsets, and at the same time, in 
order not to block the Consumer#poll method
   We cannot call ConsumerNetworkClient#poll(future)
   
   The onJoinPrepareAsyncCommitFuture variable is introduced to deal with the 
"onJoinPrepareAsyncCommitFuture.isDone() == false" situation.
   
   Maybe my consideration is complicated, this situation can be retryed by 
Consumer#poll next time.




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   2.this method `maybeAutoCommitOffsetsSync(Timer timer)` , customers will not 
use this method. There are only two places to call this method: (1) 
onJoinPrepare(...) in ConsumerCoordinator (2) close(final Timer timer) in 
ConsumerCoordinator
   

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   2.For the call to `Consumer#commitSync`, if we can’t tolerate silently 
cleaning up non-existent offsets, we still have to make sure
   The timer passed in by `commitOffsetsSync(Map offsets, Timer timer) commitOffsetsSync(Map offsets, Timer timer)` method is a controllable value, 
`max.poll.interval.ms` allows users to set by themselves, However, it is 
applied here without declaration, which is obviously out of control

##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumptio

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

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   "change its signature as ensureActiveGroup(timer, waitOnMetadata)"
   
   I think if you pass `waitOnMetadata` to `ensureActiveGroup`, it will make 
things more complicated, because
   `waitOnMetadata? timer: time.timer(0L)` will need be executed multiple times 
in `ensureCoordinatorReady` and `joinGroupIfNeeded`.




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   change its signature as ensureActiveGroup(timer, waitOnMetadata)
   I think if you pass `waitOnMetadata` to `ensureActiveGroup`, it will make 
things more complicated, because
   `waitOnMetadata? timer: time.timer(0L)` will need be executed multiple times 
in `ensureCoordinatorReady` and `joinGroupIfNeeded`.




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply.
   
   2. For the call to `Consumer#commitSync`, if we can’t tolerate silently 
cleaning up non-existent offsets, we still have to make sure that the timer 
passed in by `commitOffsetsSync(Map offsets, 
Timer timer) commitOffsetsSync(Map offsets, 
Timer timer)` method is a controllable value, `max.poll.interval.ms` allows 
users to set by themselves, However, it is applied here without declaration, 
which is obviously out of control.
   Or we can put `checkConsumedOffsetsAreValid` & `cleanUpConsumedOffsets` in 
method of `maybeAutoCommitOffsetsSync (Timer timer)` before calling 
`commitOffsetsSync`




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply.
   
   2. For the call to `Consumer#commitSync`, if we can’t tolerate silently 
cleaning up non-existent offsets, we still have to make sure that the timer 
passed in by `commitOffsetsSync(Map offsets, 
Timer timer) commitOffsetsSync(Map offsets, 
Timer timer)` method is a controllable value, `max.poll.interval.ms` allows 
users to set by themselves, However, it is applied here without declaration, 
which is obviously out of control




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   2.For the call to `Consumer#commitSync`, if we can’t tolerate silently 
cleaning up non-existent offsets, we still have to make sure
   The timer passed in by `commitOffsetsSync(Map offsets, Timer timer) commitOffsetsSync(Map offsets, Timer timer)` method is a controllable value, 
`max.poll.interval.ms` allows users to set by themselves, However, it is 
applied here without declaration, which is obviously out of control




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   




-- 
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] RivenSun2 commented on a change in pull request #11340: KAFKA-13310 : KafkaConsumer cannot jump out of the poll method, and the…

2021-10-13 Thread GitBox


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



##
File path: 
clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##
@@ -994,11 +996,16 @@ public boolean commitOffsetsSync(Map offsets,
 if (offsets.isEmpty())
 return true;
 
+boolean shouldCleanUpConsumedOffsets = 
!checkConsumedOffsetsAreValid(offsets);

Review comment:
   1. When re-join group via `maybeAutoCommitOffsetsSync`, the default 
timer is `time.timer(rebalanceTimeoutMs)`.
Because our consumers' consumption logic is sometimes heavier, we refer to 
the configuration of Kafka stream 
https://kafka.apache.org/documentation/#upgrade_10201_notable
   Set max.poll.interval.ms to Integer.MAX_VALUE
   Therefore, in the maybeAutoCommitOffsetsSync method, there will be an 
endless loop of retry submitting the offset within the Integer.MAX_VALUE time.
   The impacts are: (1)rebalance cannot be completed, group consumption 
progress is suspended; (2)offsets  commit with a high frequency, and CPU 
consumption on the broker side increases sharply
   
   2.this method `maybeAutoCommitOffsetsSync(Timer timer)` , customers will not 
use this method. There are only two places to call this method: (1) 
onJoinPrepare(...) in ConsumerCoordinator (2) close(final Timer timer) in 
ConsumerCoordinator
   




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