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 2020/11/20 06:22:17 UTC

[GitHub] [kafka] showuon opened a new pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   This improvement is side effect of this PR: https://github.com/apache/kafka/pull/6972, where we tried to fix flooding number of warning messages but never left group case. So we move the warn logs into the `maybeLeaveGroup` method, and log as INFO level. (https://github.com/apache/kafka/pull/6972/files#diff-15efe9b844f78b686393b6c2e2ad61306c3473225742caed05c7edab9a138832L1092)
   
   The INFO log is good when the leaving group is expected, ex: unsubscribe from all topics, consumer close... However, there are some cases that are unexpected, ex: polling timeout, taking too long to read the log, and these cases should be logged as WARN because some Kafka users ignore INFO messages or have the log level set to WARN. 
   
   ### 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.

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



[GitHub] [kafka] chia7712 commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @showuon thanks for this patch!


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda  @guozhangwang  , could you please help review this PR? 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda  @guozhangwang , please help review this PR. 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.

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



[GitHub] [kafka] ableegoldman commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
##########
@@ -202,7 +202,7 @@ public void requestRejoin() {
     }
 
     public void maybeLeaveGroup(String leaveReason) {
-        coordinator.maybeLeaveGroup(leaveReason);
+        coordinator.maybeLeaveGroup(leaveReason, true);

Review comment:
       The only invocation of `WorkerGroupMember#maybeLeaveGroup` in fact already does log a warning as to why instead of relying on `maybeLeaveGroup` to do so. Imo we should do something similar for the "consumer poll timeout has expired" case

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1023,9 +1023,14 @@ protected void close(Timer timer) {
     }
 
     /**
+     * Leaving the group. This method also sends LeaveGroupRequest and log {@code leaveReason} if this is dynamic members
+     * or unknown coordinator or state is not UNJOINED or this generation has a valid member id.

Review comment:
       I think it may be more useful to describe the cases where it will _not_ send a LeaveGroup and describe what this actually means (also it should have been 'and' not 'or' in the original):
   ```suggestion
        * Sends LeaveGroupRequest and logs the {@code leaveReason}, unless this member is using 
        * static  membership or is already not part of the group (ie does not have a valid member id, 
        * is in the UNJOINED state, or the coordinator is unknown).
   ```

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1386,7 +1400,7 @@ public void run() {
                                                     "the poll loop is spending too much time processing messages. " +
                                                     "You can address this either by increasing max.poll.interval.ms or by reducing " +
                                                     "the maximum size of batches returned in poll() with max.poll.records.";
-                            maybeLeaveGroup(leaveReason);
+                            maybeLeaveGroup(leaveReason, true);

Review comment:
       I think it would be simpler to just log the current `leaveReason` right here at the warn level, and then pass in a more brief description to `maybeLeaveGroup` rather than add a flag to that method just for this one 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda @guozhangwang  , please help review this PR. 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.

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



[GitHub] [kafka] mjsax commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @ableegoldman Can you have a look into this PR -- you are familiar with the consumer code.


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @ableegoldman , thank you very much! I'll ping @chia7712 when the build tests completed. He is in the same timezone with me. :)
   And @chia7712 , thanks for following up for this long lying PR. I believe some users are still suffering for this. 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.

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



[GitHub] [kafka] showuon commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1386,7 +1400,7 @@ public void run() {
                                                     "the poll loop is spending too much time processing messages. " +
                                                     "You can address this either by increasing max.poll.interval.ms or by reducing " +
                                                     "the maximum size of batches returned in poll() with max.poll.records.";
-                            maybeLeaveGroup(leaveReason);
+                            maybeLeaveGroup(leaveReason, true);

Review comment:
       Good 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.

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



[GitHub] [kafka] showuon commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1023,9 +1023,14 @@ protected void close(Timer timer) {
     }
 
     /**
+     * Leaving the group. This method also sends LeaveGroupRequest and log {@code leaveReason} if this is dynamic members
+     * or unknown coordinator or state is not UNJOINED or this generation has a valid member id.
+     *
+     * @param leaveReason the reason to leave the group for logging
+     * @param shouldWarn should log as WARN level or INFO
      * @throws KafkaException if the rebalance callback throws exception
      */
-    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
+    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason, boolean shouldWarn) throws KafkaException {

Review comment:
       Thanks, updated.




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

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



[GitHub] [kafka] showuon edited a comment on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   Failed test cases are unrelated and flaky (all passed in my local env). Thanks.
   `tests/Build/JDK 11 and Scala 2.13` --> all tests passed
   ```
       Build / JDK 15 and Scala 2.13 / kafka.network.DynamicConnectionQuotaTest.testDynamicListenerConnectionCreationRateQuota()
       Build / JDK 15 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
       Build / JDK 15 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
       Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest.testReplication()
       Build / JDK 8 and Scala 2.12 / kafka.admin.TopicCommandWithAdminClientTest.testDescribeUnderMinIsrPartitionsMixed()
   ```


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   All failed tests are flaky and unrelated.
   ```
       Build / JDK 11 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateListDeleteTopic()
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testOneWayReplicationWithAutoOffsetSync()
       Build / JDK 15 and Scala 2.13 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationSSLTest.testReplicationWithEmptyPartition()
       Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest.testReplication()
   ```


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda @guozhangwang , please help review this PR. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @chia7712 , thanks for the good suggestion! You make the change simpler!! :) Please help check again.


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda  @guozhangwang , please help review this PR. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @chia7712 , please take a look. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @chia7712 , sorry, it's been a long time, and looks like the original author and reviewer is busy, could you help check this PR? Quite straightforward to update some logging logic. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda @guozhangwang @mjsax  , please help review this PR. 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.

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



[GitHub] [kafka] showuon commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1001,9 +1001,14 @@ protected void close(Timer timer) {
     }
 
     /**
+     * Leave the group. This method also sends LeaveGroupRequest and log {@code leaveReason} if this is dynamic members
+     * or unknown coordinator or state is not UNJOINED or this generation has a valid member id.
+     *
+     * @param leaveReason the reason to leave the group for logging
+     * @param shouldWarn should log as WARN level or INFO
      * @throws KafkaException if the rebalance callback throws exception
      */
-    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
+    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason, boolean shouldWarn) throws KafkaException {

Review comment:
       Cool! And, it cannot change to `protected` method since we used this method in `KafkaConsumer`, which is in different package.




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

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



[GitHub] [kafka] chia7712 merged pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   


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

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



[GitHub] [kafka] chia7712 commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @showuon thanks for this patch!


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

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



[GitHub] [kafka] chia7712 commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
##########
@@ -203,8 +203,8 @@ public void requestRejoin() {
         coordinator.requestRejoin();
     }
 
-    public void maybeLeaveGroup(String leaveReason) {
-        coordinator.maybeLeaveGroup(leaveReason);
+    public void maybeLeaveGroup(String leaveReason, boolean shouldWarn) {

Review comment:
       It seems the `shouldWarn` is always `true` in production. Maybe we can remove the input argument from this method?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1353,12 +1363,13 @@ public void run() {
                         } else if (heartbeat.pollTimeoutExpired(now)) {
                             // the poll timeout has expired, which means that the foreground thread has stalled
                             // in between calls to poll().
-                            String leaveReason = "consumer poll timeout has expired. This means the time between subsequent calls to poll() " +
-                                                    "was longer than the configured max.poll.interval.ms, which typically implies that " +
+                            final String leaveReason = "consumer poll timeout has expired. This means the time between " +

Review comment:
       Did you revise the content? Or it is just about decoration?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1001,9 +1001,14 @@ protected void close(Timer timer) {
     }
 
     /**
+     * Leave the group. This method also sends LeaveGroupRequest and log {@code leaveReason} if this is dynamic members
+     * or unknown coordinator or state is not UNJOINED or this generation has a valid member id.
+     *
+     * @param leaveReason the reason to leave the group for logging
+     * @param shouldWarn should log as WARN level or INFO
      * @throws KafkaException if the rebalance callback throws exception
      */
-    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
+    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason, boolean shouldWarn) throws KafkaException {

Review comment:
       Most cases pass `false` so an override method  `maybeLeaveGroup(String)` can reduce the code changes. Also, `maybeLeaveGroup(String, boolean)` can be declared as a protected method. WDYT?

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1013,8 +1018,13 @@ protected void close(Timer timer) {
             state != MemberState.UNJOINED && generation.hasMemberId()) {
             // this is a minimal effort attempt to leave the group. we do not
             // attempt any resending if the request fails or times out.
-            log.info("Member {} sending LeaveGroup request to coordinator {} due to {}",
+            final String logMessage = String.format("Member %s sending LeaveGroup request to coordinator %s due to %s",

Review comment:
       'final' is unnecessary 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.

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



[GitHub] [kafka] ableegoldman commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   > I'll ping @chia7712 when the build tests completed. He is in the same timezone with me. :)
   
   Perfect. Go team


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @omkreddy @vvcephei , sorry, it's been a long time, and looks like the original author and reviewer is busy, could you help check this PR? Quite straightforward to update some logging logic. 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.

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



[GitHub] [kafka] chia7712 merged pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   


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

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda @guozhangwang , please help review this PR. I believe it impacts some users. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda @guozhangwang , please help review this PR. I believe it impacts some users. 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @abbccdda  @guozhangwang  , could you please help review this PR? 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   @ableegoldman @kkonstantine , thanks for the good comments. It makes the change smaller and simpler. Please help check again. 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.

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



[GitHub] [kafka] kkonstantine commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1023,9 +1023,14 @@ protected void close(Timer timer) {
     }
 
     /**
+     * Leaving the group. This method also sends LeaveGroupRequest and log {@code leaveReason} if this is dynamic members
+     * or unknown coordinator or state is not UNJOINED or this generation has a valid member id.
+     *
+     * @param leaveReason the reason to leave the group for logging
+     * @param shouldWarn should log as WARN level or INFO
      * @throws KafkaException if the rebalance callback throws exception
      */
-    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) {
+    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason, boolean shouldWarn) throws KafkaException {

Review comment:
       `KafkaException` is a runtime exception and therefore should only be included in the javadoc. In the method signature we include checked exceptions. 

##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -1051,6 +1061,10 @@ protected void close(Timer timer) {
         return future;
     }
 
+    public synchronized RequestFuture<Void> maybeLeaveGroup(String leaveReason) throws KafkaException {

Review comment:
       same 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.

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



[GitHub] [kafka] showuon commented on pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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


   Failed test cases are unrelated and flaky (all passed in my local env). Thanks.
   ```
       Build / JDK 15 and Scala 2.13 / kafka.network.DynamicConnectionQuotaTest.testDynamicListenerConnectionCreationRateQuota()
       Build / JDK 15 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
       Build / JDK 15 and Scala 2.13 / kafka.server.RaftClusterTest.testCreateClusterAndCreateAndManyTopicsWithManyPartitions()
       Build / JDK 8 and Scala 2.12 / org.apache.kafka.connect.mirror.integration.MirrorConnectorsIntegrationTest.testReplication()
       Build / JDK 8 and Scala 2.12 / kafka.admin.TopicCommandWithAdminClientTest.testDescribeUnderMinIsrPartitionsMixed()
   ```


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

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



[GitHub] [kafka] showuon commented on a change in pull request #9627: KAFKA-10746: Change to Warn logs when necessary to notify users

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



##########
File path: connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java
##########
@@ -203,8 +203,8 @@ public void requestRejoin() {
         coordinator.requestRejoin();
     }
 
-    public void maybeLeaveGroup(String leaveReason) {
-        coordinator.maybeLeaveGroup(leaveReason);
+    public void maybeLeaveGroup(String leaveReason, boolean shouldWarn) {

Review comment:
       Nice 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.

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