[ https://issues.apache.org/jira/browse/KAFKA-9659?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17056508#comment-17056508 ]
Guozhang Wang commented on KAFKA-9659: -------------------------------------- My search in kibana has some good news and some bad news: 1) I can confirm that the blip on leaders cause the topic-partition __consumer_offsets-35, which stores this ksql consumer group, gets migrated from broker-2 to broker-11 around 00:55:22 March 5th 2020, 00:55:22.283 controller-event-thread [Controller id=11 epoch=157] Sending become-leader LeaderAndIsr request LeaderAndIsrPartitionState(topicName='__consumer_offsets', partitionIndex=35, controllerEpoch=157, leader=11, leaderEpoch=374, isr=[11, 9, 1], zkVersion=639, replicas=[2, 9, 1, 11], observers=[], addingReplicas=[2], removingReplicas=[11], isNew=false) to broker 11 for partition __consumer_offsets-35 TRACE kafka-11 state.change.logger And then later gets back to broker-2 about three seconds later: March 5th 2020, 00:55:25.380 controller-event-thread [Controller id=11 epoch=157] Sending become-leader LeaderAndIsr request LeaderAndIsrPartitionState(topicName='__consumer_offsets', partitionIndex=35, controllerEpoch=157, leader=2, leaderEpoch=375, isr=[11, 9, 1, 2], zkVersion=641, replicas=[2, 9, 1], observers=[], addingReplicas=[], removingReplicas=[], isNew=false) to broker 2 for partition __consumer_offsets-35 TRACE kafka-11 state.change.logger The tricky part is that while broker-11 is becoming the new leader, it is also included in the `removingReplicas` which sounds quite weird, I suspect that although the new group generation is written with ack=all and the broker-2 is elected as a clean leader, the previous election of broker-11 was wonky. 2) From all the fenced entries, e.g.: March 5th 2020, 00:56:50.599 data-plane-kafka-request-handler-3 given member.id ksql-0-3-70cb05ca-4687-4c1e-9c76-7a5a94da9961 is identified as a known static member ksql-0-3,but not matching the expected member.id ksql-0-3-db32cf5a-6d18-435b-bead-82fce848e239 I can find that the "new" member.id (ksql-0-3-70cb05ca-4687-4c1e-9c76-7a5a94da9961) was actually used in the past for committing for quite some time, for example: March 4th 2020, 03:00:42.980 data-plane-kafka-network-thread-11-ListenerName(EXTERNAL)-SASL_SSL-8 Completed request:RequestHeader(apiKey=OFFSET_COMMIT, apiVersion=8, clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-d9c32901-eb60-4165-af3f-2c334ffea2b4-StreamThread-3-consumer, correlationId=7229283) -- {group_id=lkc-19kmj__confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0,generation_id=131,member_id=ksql-0-3-70cb05ca-4687-4c1e-9c76-7a5a94da9961,group_instance_id=ksql-0-3,topics=[{name=lkc-19kmj_ksql-soak-ratings-json,partitions=[{partition_index=2,committed_offset=1549665341,committed_leader_epoch=-1,committed_metadata=AQAAAXCjfPu8,_tagged_fields={}}],_tagged_fields={}}],_tagged_fields={}},response:{throttle_time_ms=0,topics=[{name=lkc-19kmj_ksql-soak-ratings-json,partitions=[{partition_index=2,error_code=0,_tagged_fields={}}],_tagged_fields={}}],_tagged_fields={}} from connection 100.109.28.58:9092-10.13.85.64:33212-567453;totalTime:5.011,requestQueueTime:0.13,localTime:0.402,remoteTime:4.351,throttleTime:0.048,responseQueueTime:0.021,sendTime:0.055,sendIoTime:0.044,securityProtocol:SASL_SSL,principal:MultiTenantPrincipal(tenantMetadata=TenantMetadata(tenantName='lkc-19kmj', clusterId='lkc-19kmj', allowDescribeBrokerConfigs=false, isSuperUser=false), user=67342),listener:EXTERNAL,clientInformation:ClientInformation(softwareName=apache-kafka-java, softwareVersion=5.5.0-ccs-SNAPSHOT) INFO kafka-11 kafka.request.logger Where as the "old" member.id cannot be found used anywhere besides the above ERROR logs, and without more logs I cannot conclude where on earth this member.id was introduced to the broker's cache at all. > Kafka Streams / Consumer configured for static membership fails on "fatal > exception: group.instance.id gets fenced" > ------------------------------------------------------------------------------------------------------------------- > > Key: KAFKA-9659 > URL: https://issues.apache.org/jira/browse/KAFKA-9659 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.5.0 > Reporter: Rohan Desai > Priority: Major > Attachments: ksql-1.logs > > > I'm running a KSQL query, which underneath is built into a Kafka Streams > application. The application has been running without issue for a few days, > until today, when all the streams threads exited with: > > > {{[ERROR] 2020-03-05 00:57:58,776 > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > org.apache.kafka.clients.consumer.internals.AbstractCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5] > Received fatal exception: group.instance.id gets fenced}} > {{[ERROR] 2020-03-05 00:57:58,776 > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > org.apache.kafka.clients.consumer.internals.AbstractCoordinator onFailure - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5] > Caught fenced group.instance.id Optional[ksql-1-2] error in heartbeat thread}} > {{[ERROR] 2020-03-05 00:57:58,776 > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > org.apache.kafka.streams.processor.internals.StreamThread run - > stream-thread > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > Encountered the following unexpected Kafka exception during processing, this > usually indicate Streams internal errors:}} > \{{ org.apache.kafka.common.errors.FencedInstanceIdException: The broker > rejected this static consumer since another consumer with the same > group.instance.id has registered with a different member.id.}}{{[INFO] > 2020-03-05 00:57:58,776 > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > org.apache.kafka.streams.processor.internals.StreamThread setState - > stream-thread > [_confluent-ksql-pksqlc-xm6g1query_CSAS_RATINGS_WITH_USER_AVERAGE_5-39e8046a-b6e6-44fd-8d6d-37cff78649bf-StreamThread-2] > State transition from RUNNING to PENDING_SHUTDOWN}} > > I've attached the KSQL and Kafka Streams logs to this ticket. Here's a > summary for one of the streams threads (instance id `ksql-1-2`): > > Around 00:56:36 the coordinator fails over from b11 to b2: > > {{[INFO] 2020-03-05 00:56:36,258 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.AbstractCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Attempt to > heartbeat failed since coordinator > b11-pkc-lzxjz.us-west-2.aws.devel.cpdev.cloud:9092 (id: 2147483636 rack: > null) is either not started or not valid.}} > {{ [INFO] 2020-03-05 00:56:36,258 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.AbstractCoordinator > markCoordinatorUnknown - [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Group > coordinator b11-pkc-lzxjz.us-west-2.aws.devel.cpdev.cloud:9092 (id: > 2147483636 rack: null) is unavailable or invalid, will attempt rediscovery}} > {{ [INFO] 2020-03-05 00:56:36,270 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.AbstractCoordinator onSuccess - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Discovered > group coordinator b2-pkc-lzxjz.us-west-2.aws.devel.cpdev.cloud:9092 (id: > 2147483645 rack: null)}} > > A few seconds later, offset commits start failing with an error indicating > the new coordinator is initializing: > > {{[WARN] 2020-03-05 00:56:39,048 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Offset > commit failed on partition ksql-soak-ratings-json-19 at offset 1825760083: > The coordinator is loading and hence can't process requests.}} > > Looking at ksql-1-2, it looks like it keeps trying to commit on the same > partition every half-second or so, which is the retry internal > (retry.backoff.ms), so it's probably stuck in the consumer's retry loop, e.g: > {{[WARN] 2020-03-05 00:56:46,616 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Offset > commit failed on partition ksql-soak-ratings-json-19 at offset 1825760083: > The coordinator is loading and hence can't process requests.}} > {{[WARN] 2020-03-05 00:56:47,175 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Offset > commit failed on partition ksql-soak-ratings-json-19 at offset 1825760083: > The coordinator is loading and hence can't process requests.}} > {{[WARN] 2020-03-05 00:56:47,742 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Offset > commit failed on partition ksql-soak-ratings-json-19 at offset 1825760083: > The coordinator is loading and hence can't process requests.}} > > This goes on until 00:56:50, when the offset commit requests fail because the > member ID doesn't match the coordinator expects: > > {{[ERROR] 2020-03-05 00:56:50,336 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Offset > commit failed on partition ksql-soak-ratings-json-19 at offset 1825760083: > The broker rejected this static consumer since another consumer with the same > group.instance.id has registered with a different member.id.}} > {{ [ERROR] 2020-03-05 00:56:50,336 > [_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2] > org.apache.kafka.clients.consumer.internals.ConsumerCoordinator handle - > [Consumer instanceId=ksql-1-2, > clientId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0-c1df9747-f353-47f1-82fd-30b97c20d038-StreamThread-2-consumer, > groupId=_confluent-ksql-pksqlc-xm6g1query_CTAS_RATINGS_BY_USER_0] Received > fatal exception: group.instance.id gets fenced}} > > So maybe the offset commit is stuck in a retry loop and not picking up the > new member ID? -- This message was sent by Atlassian Jira (v8.3.4#803005)