You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Guozhang Wang (Jira)" <ji...@apache.org> on 2020/03/21 04:29:00 UTC

[jira] [Resolved] (KAFKA-9701) Consumer could catch InconsistentGroupProtocolException during rebalance

     [ https://issues.apache.org/jira/browse/KAFKA-9701?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Guozhang Wang resolved KAFKA-9701.
----------------------------------
    Fix Version/s: 2.5.0
       Resolution: Fixed

> Consumer could catch InconsistentGroupProtocolException during rebalance
> ------------------------------------------------------------------------
>
>                 Key: KAFKA-9701
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9701
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 2.5.0
>            Reporter: Boyang Chen
>            Assignee: Boyang Chen
>            Priority: Blocker
>             Fix For: 2.5.0
>
>         Attachments: cluster.log
>
>
> The bug was due to an out-of-order handling of the SyncGroupRequest after the LeaveGroupRequest.
> The sequence of events are:
>  # The stream thread tries to rejoin the group during runOnce#poll
>  # The join group call was successful and group was waiting for sync group result
>  # Outside the poll, task producer hits FencedException, triggering a partition lost
>  # Stream thread unsubscribes and sends out an leave group, and gets the local generation wipe out 
>  # The sync group response was processed. Although it is legitimate, the local protocol type becomes null in this case
>  # 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.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)