[ https://issues.apache.org/jira/browse/KAFKA-10643?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17230181#comment-17230181 ]
Eran Levy edited comment on KAFKA-10643 at 11/11/20, 7:39 PM: -------------------------------------------------------------- It sometimes happens that the rebalances taking long time, unfortunately it happened today again (11/11 13:20), I really dont understand why it happened again after digging the logs. At least I have a full client log to share with you, the issue started with d-9 (was up since 9/11 without any special problems I think) client removal cause of heartbeat - started with this broker log: [2020-11-11 13:24:21,664] INFO [GroupCoordinator 1]: Preparing to rebalance group d in state PreparingRebalance with old generation 13235 (__consumer_offsets-45) (reason: removing member d-9-1-b6e95dc4-bcf1-4789-aaae-db3e8dbad1d6 on heartbeat expiration) (kafka.coordinator.group.GroupCoordinator) then it followed with all the other client failing, around 5 mins later 13:29: Nov 11 13:29:08 ip-1-1-1-1 dhclient[2669]: XMT: Solicit on eth0, interval 124650ms. Nov 11 13:29:17 ip-10-32-4-201 kafka-server-start.sh: [2020-11-11 13:29:17,501] INFO [GroupCoordinator 1]: Member d-11-7-c62e9c5a-7f68-4f2d-8870-287bd7524a2c in group d has failed, removing it from the group (kafka.coordinator.group.GroupCoordinator) client logs of d-9 attached the first one that failed, what worries the most is this log that after a really long time, the rebalance didnt finish: {code:java} "November 11th 2020, 13:37:12.506","{""timestamp"":""2020-11-11T13:37:12.505Z"",""level"":""WARN"",""thread"":""d-8fc0f3fe-55b6-4494-b63e-8d889e96117d-StreamThread-3"",""logger"":""org.apache.kafka.streams.processor.internals.StreamThread"",""message"":""stream-thread [d-8fc0f3fe-55b6-4494-b63e-8d889e96117d-StreamThread-3] Detected that the thread is being fenced. This implies that this thread missed a rebalance and dropped out of the consumer group. Will close out all assigned tasks and rejoin the consumer group."",""context"":""default"",""exception"":""org.apache.kafka.streams.errors.TaskMigratedException: Consumer committing offsets failed, indicating the corresponding thread is no longer part of the group; it means all tasks belonging to this thread should be migrated.\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commitOffsetsOrTransaction(TaskManager.java:1009)\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commit(TaskManager.java:962)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:851)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:714)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:551)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:510)\nCaused by: org.apache.kafka.clients.consumer.CommitFailedException: Offset commit cannot be completed since the consumer is not part of an active group for auto partition assignment; it is likely that the consumer was kicked out of the group.\n\tat org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:1134)\n\tat org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:999)\n\tat org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1504)\n\tat org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1452)\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commitOffsetsOrTransaction(TaskManager.java:1007)\n\t... 5 common frames omitted\n""}","d","d-9" {code} Hope you might be able to shed some light here..because its very very strange. [^client-d-9-11-11-2020.csv] was (Author: eran-levy): It sometimes happens that the rebalances taking long time, unfortunately it happened today again (11/11 13:20), I really dont understand why it happened again after digging the logs. At least I have a full client log to share with you, the issue started with d-9 client removal cause of heartbeat - started with this broker log: [2020-11-11 13:24:21,664] INFO [GroupCoordinator 1]: Preparing to rebalance group d in state PreparingRebalance with old generation 13235 (__consumer_offsets-45) (reason: removing member d-9-1-b6e95dc4-bcf1-4789-aaae-db3e8dbad1d6 on heartbeat expiration) (kafka.coordinator.group.GroupCoordinator) then it followed with all the other client failing, around 5 mins later 13:29: Nov 11 13:29:08 ip-1-1-1-1 dhclient[2669]: XMT: Solicit on eth0, interval 124650ms. Nov 11 13:29:17 ip-10-32-4-201 kafka-server-start.sh: [2020-11-11 13:29:17,501] INFO [GroupCoordinator 1]: Member d-11-7-c62e9c5a-7f68-4f2d-8870-287bd7524a2c in group d has failed, removing it from the group (kafka.coordinator.group.GroupCoordinator) client logs of d-9 attached the first one that failed, what worries the most is this log that after a really long time, the rebalance didnt finish: {code:java} "November 11th 2020, 13:37:12.506","{""timestamp"":""2020-11-11T13:37:12.505Z"",""level"":""WARN"",""thread"":""d-8fc0f3fe-55b6-4494-b63e-8d889e96117d-StreamThread-3"",""logger"":""org.apache.kafka.streams.processor.internals.StreamThread"",""message"":""stream-thread [d-8fc0f3fe-55b6-4494-b63e-8d889e96117d-StreamThread-3] Detected that the thread is being fenced. This implies that this thread missed a rebalance and dropped out of the consumer group. Will close out all assigned tasks and rejoin the consumer group."",""context"":""default"",""exception"":""org.apache.kafka.streams.errors.TaskMigratedException: Consumer committing offsets failed, indicating the corresponding thread is no longer part of the group; it means all tasks belonging to this thread should be migrated.\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commitOffsetsOrTransaction(TaskManager.java:1009)\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commit(TaskManager.java:962)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:851)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:714)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:551)\n\tat org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:510)\nCaused by: org.apache.kafka.clients.consumer.CommitFailedException: Offset commit cannot be completed since the consumer is not part of an active group for auto partition assignment; it is likely that the consumer was kicked out of the group.\n\tat org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.sendOffsetCommitRequest(ConsumerCoordinator.java:1134)\n\tat org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.commitOffsetsSync(ConsumerCoordinator.java:999)\n\tat org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1504)\n\tat org.apache.kafka.clients.consumer.KafkaConsumer.commitSync(KafkaConsumer.java:1452)\n\tat org.apache.kafka.streams.processor.internals.TaskManager.commitOffsetsOrTransaction(TaskManager.java:1007)\n\t... 5 common frames omitted\n""}","d","d-9" {code} Hope you might be able to shed some light here..because its very very strange. [^client-d-9-11-11-2020.csv] > Static membership - repetitive PreparingRebalance with updating metadata for > member reason > ------------------------------------------------------------------------------------------ > > Key: KAFKA-10643 > URL: https://issues.apache.org/jira/browse/KAFKA-10643 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.6.0 > Reporter: Eran Levy > Priority: Major > Attachments: broker-4-11.csv, client-4-11.csv, > client-d-9-11-11-2020.csv > > > Kafka streams 2.6.0, brokers version 2.6.0. Kafka nodes are healthy, kafka > streams app is healthy. > Configured with static membership. > Every 10 minutes (I assume cause of topic.metadata.refresh.interval.ms), I > see the following group coordinator log for different stream consumers: > INFO [GroupCoordinator 2]: Preparing to rebalance group **--**-stream in > state PreparingRebalance with old generation 12244 (__consumer_offsets-45) > (reason: Updating metadata for member > ****-stream-11-1-013edd56-ed93-4370-b07c-1c29fbe72c9a) > (kafka.coordinator.group.GroupCoordinator) > and right after that the following log: > INFO [GroupCoordinator 2]: Assignment received from leader for group > **-**-stream for generation 12246 (kafka.coordinator.group.GroupCoordinator) > > Looked a bit on the kafka code and Im not sure that I get why such a thing > happening - is this line described the situation that happens here re the > "reason:"?[https://github.com/apache/kafka/blob/7ca299b8c0f2f3256c40b694078e422350c20d19/core/src/main/scala/kafka/coordinator/group/GroupCoordinator.scala#L311] > I also dont see it happening too often in other kafka streams applications > that we have. > The only thing suspicious that I see around every hour that different pods of > that kafka streams application throw this exception: > {"timestamp":"2020-10-25T06:44:20.414Z","level":"INFO","thread":"**-**-stream-94561945-4191-4a07-ac1b-07b27e044402-StreamThread-1","logger":"org.apache.kafka.clients.FetchSessionHandler","message":"[Consumer > > clientId=**-**-stream-94561945-4191-4a07-ac1b-07b27e044402-StreamThread-1-restore-consumer, > groupId=null] Error sending fetch request (sessionId=34683236, epoch=2872) > to node > 3:","context":"default","exception":"org.apache.kafka.common.errors.DisconnectException: > null\n"} > I came across this strange behaviour after stated to investigate a strange > stuck rebalancing state after one of the members left the group and caused > the rebalance to stuck - the only thing that I found is that maybe because > that too often preparing to rebalance states, the app might affected of this > bug - KAFKA-9752 ? > I dont understand why it happens, it wasn't before I applied static > membership to that kafka streams application (since around 2 weeks ago). > Will be happy if you can help me > > -- This message was sent by Atlassian Jira (v8.3.4#803005)