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

Consumer could catch InconsistentGroupProtocolException during rebalance

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 2.5.0
    • 2.5.0
    • None
    • None

    Description

      The bug was due to an out-of-order handling of the SyncGroupRequest after the LeaveGroupRequest.

      The sequence of events are:

      1. The stream thread tries to rejoin the group during runOnce#poll
      2. The join group call was successful and group was waiting for sync group result
      3. Outside the poll, task producer hits FencedException, triggering a partition lost
      4. Stream thread unsubscribes and sends out an leave group, and gets the local generation wipe out 
      5. The sync group response was processed. Although it is legitimate, the local protocol type becomes null in this case
      6. The sync group response hits the protocol type mismatch fatal exception

       

      [2020-03-20T*10:40:08-07:00*] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:08,754] INFO [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] [Consumer clientId=stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-consumer, groupId=stream-soak-test] (Re-)joining group (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)

      [2020-03-20T*10:40:11-07:00*] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:11,152] ERROR [kafka-producer-network-thread | stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-0_1-producer] stream-thread [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] task [0_1] Error encountered sending record to topic network-id-repartition for task 0_1 due to:

      [2020-03-20T10:40:11-07:00] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) org.apache.kafka.common.errors.ProducerFencedException: Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker.

      [2020-03-20T10:40:12-07:00] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:12,048] INFO [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] stream-thread [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] at state RUNNING: partitions [logs.json.kafka-1, node-name-repartition-1, logs.json.zookeeper-1, logs.kubernetes-1, windowed-node-counts-1, logs.operator-1, logs.syslog-1] lost due to missed rebalance.

              lost active tasks: []

              lost assigned standby tasks: []

      (org.apache.kafka.streams.processor.internals.StreamThread)

       

      [2020-03-20T*10:40:12-07:00*] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:12,048] INFO [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] [Consumer clientId=stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-consumer, groupId=stream-soak-test] Member stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-consumer-34c2198b-5bdd-470b-ae50-30a39873edab sending LeaveGroup request to coordinator ip-172-31-18-29.us-west-2.compute.internal:9092 (id: 2147482644 rack: null) due to the consumer unsubscribed from all topics (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)

      [2020-03-20T10:40:12-07:00] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:12,048] INFO [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] [Consumer clientId=stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-consumer, groupId=stream-soak-test] Unsubscribed all topics or patterns and assigned partitions (org.apache.kafka.clients.consumer.KafkaConsumer)

      [2020-03-20T10:40:17-07:00] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:16,972] ERROR [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] [Consumer clientId=stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1-consumer, groupId=stream-soak-test] SyncGroup failed due to inconsistent Protocol Name, received stream but expected null (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)

      [2020-03-20T10:40:17-07:00] (streams-soak-trunk-eos_soak_i-01629239fa39901b4_streamslog) [2020-03-20 17:40:16,973] ERROR [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] stream-thread [stream-soak-test-f7392d33-55d7-484f-8b72-578e22fead96-StreamThread-1] Encountered the following exception during processing and the thread is going to shut down:  (org.apache.kafka.streams.processor.internals.StreamThread)

       

      ----------- Original Exception -------------

      INFO log shows that we accidentally hit an unexpected inconsistent group protocol exception:

      [2020-03-10T17:16:53-07:00] (streams-soak-2-5-eos-broker-2-5_soak_i-00067445452c82fe8_streamslog) [2020-03-11 *00:16:53,382*] INFO [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1] stream-client [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949] State transition from REBALANCING to RUNNING (org.apache.kafka.streams.KafkaStreams)

       

      [2020-03-10T17:16:53-07:00] (streams-soak-2-5-eos-broker-2-5_soak_i-00067445452c82fe8_streamslog) [2020-03-11 *00:16:53,384*] WARN [kafka-producer-network-thread | stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1-0_1-producer] stream-thread [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1] task [0_1] Error sending record to topic node-name-repartition due to Producer attempted an operation with an old epoch. Either there is a newer producer with the same transactionalId, or the producer's transaction has been expired by the broker.; No more records will be sent and no more offsets will be recorded for this task.

       

       

      [2020-03-10T17:16:53-07:00] (streams-soak-2-5-eos-broker-2-5_soak_i-00067445452c82fe8_streamslog) [2020-03-11 *00:16:53,521*] INFO [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1] [Consumer clientId=stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1-consumer, groupId=stream-soak-test] Member stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1-consumer-d1c3c796-0bfb-4c1c-9fb4-5a807d8b53a2 sending LeaveGroup request to coordinator ip-172-31-20-215.us-west-2.compute.internal:9092 (id: 2147482646 rack: null) due to the consumer unsubscribed from all topics (org.apache.kafka.clients.consumer.internals.AbstractCoordinator)

       

      [2020-03-10T17:16:54-07:00] (streams-soak-2-5-eos-broker-2-5_soak_i-00067445452c82fe8_streamslog) [2020-03-11 *00:16:53,798*] ERROR [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1] stream-thread [stream-soak-test-d3da8597-c371-450e-81d9-72aea6a26949-StreamThread-1] Encountered the following unexpected Kafka exception during processing, this usually indicate Streams internal errors: (org.apache.kafka.streams.processor.internals.StreamThread)

      [2020-03-10T17:16:54-07:00] (streams-soak-2-5-eos-broker-2-5_soak_i-00067445452c82fe8_streamslog) org.apache.kafka.common.errors.InconsistentGroupProtocolException: The group member's supported protocols are incompatible with those of existing members or first group member tried to join with empty protocol type or empty protocol list.

       

      Potentially needs further log to understand this.

      Attachments

        1. cluster.log
          59 kB
          John Roesler

        Activity

          People

            bchen225242 Boyang Chen
            bchen225242 Boyang Chen
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: