You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/12/30 10:14:20 UTC

[GitHub] [kafka] showuon opened a new pull request #11631: KAFKA-13563: ensure coordinator ready for each consumer poll, to hand…

showuon opened a new pull request #11631:
URL: https://github.com/apache/kafka/pull/11631


   …le FindCoordinatorFuture correctly
   
   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1029598488


   @guozhangwang @ableegoldman @hachikuji , please take a look when available. This issue blocks some users when upgrading Kafka-clients. I think this should be fixed soon. 
   Thank you.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141737



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -192,6 +192,9 @@
     private final String partitionLost = "Hit partition lost ";
 
     private final Collection<TopicPartition> singleTopicPartition = Collections.singleton(new TopicPartition(topic, 0));
+    private final Time time = new MockTime();
+    private final SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
+    private final ConsumerPartitionAssignor assignor = new RoundRobinAssignor();

Review comment:
       side cleanup: create global variables to share in tests.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1029600849


   @showuon Sorry for getting late on this -- I thought it was not ready since the title still has `WIP` in it. I will re-title and continue reviewing 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r776731023



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -517,15 +517,13 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
                 }
             }
         } else {
-            // For manually assigned partitions, if there are no ready nodes, await metadata.
+            // For manually assigned partitions, if coordinator is unknown, make sure we lookup one and await metadata.
             // If connections to all nodes fail, wakeups triggered while attempting to send fetch
             // requests result in polls returning immediately, causing a tight loop of polls. Without
             // the wakeup, poll() with no channels would block for the timeout, delaying re-connection.
-            // awaitMetadataUpdate() initiates new connections with configured backoff and avoids the busy loop.
-            // When group management is used, metadata wait is already performed for this scenario as
-            // coordinator is unknown, hence this check is not required.
-            if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
-                client.awaitMetadataUpdate(timer);
+            // awaitMetadataUpdate() in ensureCoordinatorReady initiates new connections with configured backoff and avoids the busy loop.
+            if (coordinatorUnknown() && !ensureCoordinatorReady(timer)) {
+                return false;

Review comment:
       Before the change, we will wait for the metadata update if no nodes available, to avoid busy loop when in non consumer group mode. After the change, we did as the group management did, to call `ensureCoordinatorReady` when coordinator unknown. And in `ensureCoordinatorReady`. This way, we can also make sure to handle the `FindCoordinatorFuture` well (and clear it) inside `ensureCoordinatorReady`. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on a change in pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r778549345



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -517,15 +517,13 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
                 }
             }
         } else {
-            // For manually assigned partitions, if there are no ready nodes, await metadata.
+            // For manually assigned partitions, if coordinator is unknown, make sure we lookup one and await metadata.
             // If connections to all nodes fail, wakeups triggered while attempting to send fetch
             // requests result in polls returning immediately, causing a tight loop of polls. Without
             // the wakeup, poll() with no channels would block for the timeout, delaying re-connection.
-            // awaitMetadataUpdate() initiates new connections with configured backoff and avoids the busy loop.
-            // When group management is used, metadata wait is already performed for this scenario as
-            // coordinator is unknown, hence this check is not required.
-            if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
-                client.awaitMetadataUpdate(timer);
+            // awaitMetadataUpdate() in ensureCoordinatorReady initiates new connections with configured backoff and avoids the busy loop.
+            if (coordinatorUnknown() && !ensureCoordinatorReady(timer)) {
+                return false;

Review comment:
       I'm wondering if this fix is a bit overkill, e.g. for those consumers who do not even set the group ID and do not rely on the coordinator for anything, the `coordinatorUnknown()` would always return true while `ensureCoordinatorReady` would send a discover coordinator request with `null` group id. 




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon edited a comment on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon edited a comment on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1005407357


   @guozhangwang , thanks for your comment. Answering your question below:
   
   > I'm wondering if this fix is a bit overkill, e.g. for those consumers who do not even set the group ID and do not rely on the coordinator for anything, the coordinatorUnknown() would always return true while ensureCoordinatorReady would send a discover coordinator request with null group id.
   
   No, if consumer doesn't provide group id config value (default is null), we won't create `consumerCoordinator` in the consumer. That is, if the group id is provided, it's either with consumer group management (via Consumer#subscribe), or with manual assignment (via Consumer#assign) with offset commit enabled. 
   
   REF: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L775
   
   And before the PR, we will wait for the metadata update if no nodes available, to avoid busy loop when in non consumer group mode.
   
   ```java
   if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
       client.awaitMetadataUpdate(timer);
   ```
   
   After the change, we did as the group management did, to call `ensureCoordinatorReady` when coordinator unknown. And in `ensureCoordinatorReady`. This way, we can also make sure to handle the `FindCoordinatorFuture` well (and clear it) inside `ensureCoordinatorReady`.
   
   Does that make sense?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141328



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -660,15 +645,12 @@ public void verifyDeprecatedPollDoesNotTimeOutDuringMetadataUpdate() {
 
     @Test
     public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() {
-        Time time = new MockTime();
-        SubscriptionState subscription = new SubscriptionState(new LogContext(), OffsetResetStrategy.EARLIEST);
         ConsumerMetadata metadata = createMetadata(subscription);
         MockClient client = new MockClient(time, metadata);
 
         initMetadata(client, Collections.singletonMap(topic, 1));
-        ConsumerPartitionAssignor assignor = new RoundRobinAssignor();
 
-        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, groupInstanceId);
+        KafkaConsumer<String, String> consumer = newConsumer(time, client, subscription, metadata, assignor, true, null, groupInstanceId, false);

Review comment:
       This test case is to test manual assignment without storing offsets. So we should not create `groupID` (the third parameter from the end)
   
   test added for this issue: https://issues.apache.org/jira/browse/KAFKA-4034




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1030826032


   @guozhangwang , this PR is good for review now. I've fixed broken tests and added tests. Thank you.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang merged pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
guozhangwang merged pull request #11631:
URL: https://github.com/apache/kafka/pull/11631


   


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1013598333


   @guozhangwang @ableegoldman , please take a look when available. 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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141507



##########
File path: core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
##########
@@ -1037,7 +1037,8 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WildcardHost, READ, ALLOW)), topicResource)
 
     // in this case, we do an explicit seek, so there should be no need to query the coordinator at all
-    val consumer = createConsumer()
+    // remove the group.id config to avoid coordinator created
+    val consumer = createConsumer(configsToRemove = List(ConsumerConfig.GROUP_ID_CONFIG))

Review comment:
       Same as above.
   
   
   
   This test case is to test manual assignment without storing offsets. So we should not create groupID (the third parameter from the end)
   
   test added for this issue: https://issues.apache.org/jira/browse/KAFKA-4034
   




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141904



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -488,7 +492,7 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
             // Always update the heartbeat last poll time so that the heartbeat thread does not leave the
             // group proactively due to application inactivity even if (say) the coordinator cannot be found.
             pollHeartbeat(timer.currentTimeMs());
-            if (coordinatorUnknown() && !ensureCoordinatorReady(timer)) {
+            if (coordinatorUnknownAndUnready(timer)) {

Review comment:
       > Could we move the whole block before the if-else block and also update the related comments?
   
   We can't because we should always lookup the coordinator after heartbeat poll to avoid the heartbeat timeout in group management case.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141904



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -488,7 +492,7 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
             // Always update the heartbeat last poll time so that the heartbeat thread does not leave the
             // group proactively due to application inactivity even if (say) the coordinator cannot be found.
             pollHeartbeat(timer.currentTimeMs());
-            if (coordinatorUnknown() && !ensureCoordinatorReady(timer)) {
+            if (coordinatorUnknownAndUnready(timer)) {

Review comment:
       We should always lookup the coordinator after heartbeat poll to avoid the heartbeat timeout.




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800141430



##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2641,14 +2565,16 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
         ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time,
                 retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs);
 
-        GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs,
+        ConsumerCoordinator consumerCoordinator = null;
+        if (groupId != null) {
+            GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs,

Review comment:
       we should not create `consumerCoordinator` when groupID is null




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1030935640


   Merged to trunk, thanks @showuon ! Also cherry-picked to 3.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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1029613788


   @guozhangwang , thanks for your comments. Sorry for confusing you! And it makes sense to remove the `KIP` out from the PR title. 
   I'll let you know when I complete the PR. Thank you.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon edited a comment on pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon edited a comment on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1029613788


   @guozhangwang , thanks for your comments. Sorry for confusing you! And it makes sense to remove the `WIP` out from the PR title. 
   I'll let you know when I complete the PR. Thank you.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1005407357


   @guozhangwang , answer your question below:
   
   > I'm wondering if this fix is a bit overkill, e.g. for those consumers who do not even set the group ID and do not rely on the coordinator for anything, the coordinatorUnknown() would always return true while ensureCoordinatorReady would send a discover coordinator request with null group id.
   
   No, if consumer doesn't provide group id config value (default is null), we won't create `consumerCoordinator` in the consumer. That is, if the group id is provided, it's either with consumer group management (via Consumer#subscribe), or with manual assignment (via Consumer#assign) with offset commit enabled. 
   
   REF: https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java#L775
   
   And before the PR, we will wait for the metadata update if no nodes available, to avoid busy loop when in non consumer group mode.
   
   ```java
   if (metadata.updateRequested() && !client.hasReadyNodes(timer.currentTimeMs())) {
       client.awaitMetadataUpdate(timer);
   ```
   
   After the change, we did as the group management did, to call `ensureCoordinatorReady` when coordinator unknown. And in `ensureCoordinatorReady`. This way, we can also make sure to handle the `FindCoordinatorFuture` well (and clear it) inside `ensureCoordinatorReady`.
   
   Does that make sense?


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] guozhangwang commented on a change in pull request #11631: KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
guozhangwang commented on a change in pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#discussion_r800245635



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
##########
@@ -488,7 +492,7 @@ public boolean poll(Timer timer, boolean waitForJoinGroup) {
             // Always update the heartbeat last poll time so that the heartbeat thread does not leave the
             // group proactively due to application inactivity even if (say) the coordinator cannot be found.
             pollHeartbeat(timer.currentTimeMs());
-            if (coordinatorUnknown() && !ensureCoordinatorReady(timer)) {
+            if (coordinatorUnknownAndUnready(timer)) {

Review comment:
       Makes sense.

##########
File path: clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
##########
@@ -2641,14 +2565,16 @@ private FetchResponse fetchResponse(TopicPartition partition, long fetchOffset,
         ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time,
                 retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs);
 
-        GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs,
+        ConsumerCoordinator consumerCoordinator = null;
+        if (groupId != null) {
+            GroupRebalanceConfig rebalanceConfig = new GroupRebalanceConfig(sessionTimeoutMs,

Review comment:
       Nice cleanup!




-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on pull request #11631: [WIP] KAFKA-13563: clear FindCoordinatorFuture for non consumer group mode

Posted by GitBox <gi...@apache.org>.
showuon commented on pull request #11631:
URL: https://github.com/apache/kafka/pull/11631#issuecomment-1003034974


   @guozhangwang @ableegoldman , could you please take a look to see if this fix makes sense to you? I'll fix the broken tests after your first review. Thank you.


-- 
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-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org