You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Yining Liu (JIRA)" <ji...@apache.org> on 2018/11/29 08:20:00 UTC

[jira] [Commented] (KAFKA-6681) Two instances of kafka consumer reading the same partition within a consumer group

    [ https://issues.apache.org/jira/browse/KAFKA-6681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16702842#comment-16702842 ] 

Yining Liu commented on KAFKA-6681:
-----------------------------------

We are facing the similar issue when we use MirrorMaker new consumer (0.10.2.1). And this issue was fixed on 0.11 (KAFKA-5154)
 The situation is:
 # many consumers consuming same topic
 # network broken, and recovered
 # one consumer group coordinator dead, and never came back
 # consumer.commitSync() got failure due to ConsumerCoordinator#sendOffsetCommitRequest()'s failure (coordinator's state is not stable->generation == null)
 # on consumer.poll()->ConsumerCoordinator#poll(), needRejoin() returns false, so it won't call ensureActiveGroup(). The coordinator never re-join group again. But consumer.poll() can still get messages.
 # on MirrorMaker, consumer.commitSync() always fail, we can not see this consumer instance in consumer group, consumer.poll() is still able to get messages. We receive lots of log like "Failed to commit offsets because the consumer group has rebalanced and assigned partitions to ....."
 # other consumer is assigned to these partitions. some partitions is consumed by two consumers. MM generates a lot of duplicate data.

We are able to reproduce (stable) this issue on dev by following steps:
 # start two MMs consuming same topic (4 partitions)
 # find coordinator(a broker) of this consumer group
 # simulate network connect issue from one consumer to coordinator by tc. (tc qdisc add dev eth0 parent 1:2 handle 10: netem delay 1ms 10000ms 30% loss 20% 20%)
 # kill -STOP coordinator's broker process
 # wait 5 mins

After we move [https://github.com/apache/kafka/commit/1b16acaaa181ceb214d84e70b8ddc146af9c0c5c] into our 0.10 code, this issue never happen again.

> Two instances of kafka consumer reading the same partition within a consumer group
> ----------------------------------------------------------------------------------
>
>                 Key: KAFKA-6681
>                 URL: https://issues.apache.org/jira/browse/KAFKA-6681
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, consumer
>    Affects Versions: 0.10.2.1
>            Reporter: Narayan Periwal
>            Priority: Critical
>         Attachments: server-1.log, server-2.log
>
>
> We have seen this issue with the Kafka consumer, the new library that got introduced in 0.9
> With this new client, the group management is done by kafka coordinator, which is one of the kafka broker.
> We are using Kafka broker 0.10.2.1 and consumer client version is also 0.10.2.1 
> The issue that we have faced is that, after rebalancing, some of the partitions gets consumed by 2 instances within a consumer group, leading to duplication of the entire partition data. Both the instances continue to read until the next rebalancing, or the restart of those clients. 
> It looks like that a particular consumer goes on fetching the data from a partition, but the broker is not able to identify this "stale" consumer instance. 
> We have hit this twice in production. Please look at it the earliest. 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)