Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-9659

Kafka Streams / Consumer configured for static membership fails on "fatal exception: group.instance.id gets fenced"

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.5.0
    • 2.6.0
    • streams
    • None

    Description

      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?

      Attachments

        1. ksql-1.logs
          1.48 MB
          Rohan Desai

        Issue Links

          Activity

            People

              guozhang Guozhang Wang
              rohanpd Rohan Desai
              Votes:
              0 Vote for this issue
              Watchers:
              10 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: