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/09/30 09:50:04 UTC

[GitHub] [kafka] chia7712 commented on a change in pull request #9354: KAFKA-10134 Follow-up: Set the re-join flag in heartbeat failure

chia7712 commented on a change in pull request #9354:
URL: https://github.com/apache/kafka/pull/9354#discussion_r497383527



##########
File path: clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
##########
@@ -948,7 +948,7 @@ private synchronized void resetStateAndRejoin() {
     synchronized void resetGenerationOnResponseError(ApiKeys api, Errors error) {
         log.debug("Resetting generation after encountering {} from {} response and requesting re-join", error, api);
 
-        resetState();
+        resetStateAndRejoin();

Review comment:
       @dajac I feel this patch can fix https://issues.apache.org/jira/browse/KAFKA-8266
   
   The test case in https://issues.apache.org/jira/browse/KAFKA-8266 restarts all brokers to activate new configs. However, the data of group (__consumer_offsets) may get lost if all brokers are killed too quick (the log folder is changed when restarting broker so the restarting broker has to fetch data from other brokers). The heartbeat of running consumer will encounter ```UNKNOWN_MEMBER_ID``` in sending heartbeat request since the group data is gone. Without this patch, the heartbeat thread is disabled and the state is in ```UNJOINED``` but the ```rejoinNeeded``` is still false. In short, the consumer is not going to rejoin group so we can't see expected error.




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