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

2018-11-29 Thread Yining Liu (JIRA)


[ 
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 1ms 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)


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

2018-11-29 Thread Narayan Periwal (JIRA)


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

Narayan Periwal commented on KAFKA-6681:


Thanks for letting us know [~lyn610]

> 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)


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

2018-03-20 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~tedyu], Attached the server side logs. Could not retrieve the consumer side 
logs as it has hit the retention, I am again trying to reproduce this in our QA 
setup. See if the server side logs is of any help.

The server side logs correspond to those node in which the under replicated 
metrics spiked during this time

There is no logs in the controller.log file during this time.

> 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: 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. 
> During this time, we also see the underreplicated partition metrics spiking. 
> We have hit this twice in production. Please look at it the earliest. 



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


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

2018-03-20 Thread Ted Yu (JIRA)

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

Ted Yu commented on KAFKA-6681:
---

{code}
[2018-03-14 05:04:13,231] ERROR [ReplicaFetcherThread-0-15], Error for 
partition [adhoc_network_click_ams1_ams1,5] to broker 
15:org.apache.kafka.common.errors.
NotLeaderForPartitionException: This server is not the leader for that 
topic-partition. (kafka.server.ReplicaFetcherThread)
[2018-03-14 05:04:13,233] ERROR [ReplicaFetcherThread-7-17], Error for 
partition [cas_rr,21] to broker 
17:org.apache.kafka.common.errors.NotLeaderForPartitionException: This server 
isnot the leader for that topic-partition. 
(kafka.server.ReplicaFetcherThread)
{code}
Did the partition being read by two consumers appear in the ERROR log ?

Thanks

> 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: 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. 
> During this time, we also see the underreplicated partition metrics spiking. 
> We have hit this twice in production. Please look at it the earliest. 



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


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

2018-03-20 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com]

The partition read by the two consumers did not appear in any of the kafka 
broker logs. Partition number 1 of the topic renderCpmAms1 was being consumed 
by two consumer instances within a consumer group

The following are the log lines in the server logs related to this topic
{noformat}
[2018-03-14 05:01:53,456] INFO Partition [renderCpmAms1,10] on broker 1: 
Shrinking ISR for partition [renderCpmAms1,10] from 1,2,3 to 1,3 
(kafka.cluster.Partition)
[2018-03-14 05:02:14,122] INFO Partition [renderCpmAms1,10] on broker 1: 
Expanding ISR for partition renderCpmAms1-10 from 1,3 to 1,3,2 
(kafka.cluster.Partition)
[2018-03-14 05:01:52,376] INFO Partition [renderCpmAms1,9] on broker 15: 
Shrinking ISR for partition [renderCpmAms1,9] from 2,15,1 to 15,1 
(kafka.cluster.Partition)
[2018-03-14 05:02:14,193] INFO Partition [renderCpmAms1,9] on broker 15: 
Expanding ISR for partition renderCpmAms1-9 from 15,1 to 15,1,2 
(kafka.cluster.Partition)

[2018-03-14 05:02:17,510] INFO Partition [renderCpmAms1,11] on broker 2: 
Shrinking ISR for partition [renderCpmAms1,11] from 2,4,3 to 2,4 
(kafka.cluster.Partition)
[2018-03-14 05:02:17,530] INFO Partition [renderCpmAms1,11] on broker 2: Cached 
zkVersion [171] not equal to that in zookeeper, skip updating ISR 
(kafka.cluster.Partition)
{noformat}

Wondering, if this error or log line has any corelation with the issue.
{noformat}
[2018-03-14 05:02:17,530] INFO Partition [renderCpmAms1,11] on broker 2: Cached 
zkVersion [171] not equal to that in zookeeper, skip updating ISR 
(kafka.cluster.Partition)
{noformat}


> 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: 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. 
> During this time, we also see the underreplicated partition metrics spiking. 
> We have hit this twice in production. Please look at it the earliest. 



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


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

2018-03-26 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com], any update on this?

> 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: 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. 
> During this time, we also see the underreplicated partition metrics spiking. 
> We have hit this twice in production. Please look at it the earliest. 



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


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

2018-03-26 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com], 
We faced yet another such issue, on server side we found these logs in this case

{noformat}
[2018-03-23 18:59:16,560] INFO [GroupCoordinator 6]: Stabilized group 
prod-m10n-event-batcher-billablebeaconams1 generation 6 
(kafka.coordinator.GroupCoordinator)
[2018-03-23 18:59:46,561] INFO [GroupCoordinator 6]: Preparing to restabilize 
group prod-m10n-event-batcher-billablebeaconams1 with old generation 6 
(kafka.coordinator.GroupCoordinator)
[2018-03-23 18:59:46,833] INFO [GroupCoordinator 6]: Stabilized group 
prod-m10n-event-batcher-billablebeaconams1 generation 7 
(kafka.coordinator.GroupCoordinator)
{noformat}

> 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: 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. 
> During this time, we also see the underreplicated partition metrics spiking. 
> We have hit this twice in production. Please look at it the earliest. 



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


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

2018-03-29 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com],
We had one more occurence of the above issue. The topic had 1 partition and 
there were 4 consumers for it (all with the same consumer group name). 
Initially, as expected, only one if the consumer was reading from that 
partition and the others were simply doing nothing. We had an issue with our 
kafka cluster, due to which the entire cluster went down. When the cluster got 
up, after that I see all the 4 consumers reading that single partition of the 
topic, which was strange. 

For that topic, this is logs from the coordinator on the server side for that 
consumer group

{noformat}
[2018-03-27 23:06:49,113] INFO [GroupCoordinator 8]: Loading group metadata for 
testgroup with generation 63 (kafka.coordinator.GroupCoordinator)
[2018-03-27 23:06:52,687] INFO [GroupCoordinator 8]: Preparing to restabilize 
group testgroup with old generation 63 (kafka.coordinator.GroupCoordinator)
[2018-03-27 23:06:52,688] INFO [GroupCoordinator 8]: Stabilized group testgroup 
generation 64 (kafka.coordinator.GroupCoordinator)
[2018-03-27 23:06:52,916] INFO [GroupCoordinator 8]: Assignment received from 
leader for group testgroup for generation 64 
(kafka.coordinator.GroupCoordinator)
{noformat}

On the consumer side, the client-1 that was already reading that partition, on 
that we see the rebalancing getting triggered, both the callbacks 
onPartitionsRevoked and onPartitionsAssigned were invoked, while on client-2, 
none of these callbacks were invoked, however, still it started consuming the 
data from the partition, from there on. We saw the following exception in the 
client-2 logs, occuring 4 times with a gap of 1 to 2 seconds
{noformat}
27 Mar 2018 23:06:42.307 ERROR [testgroup:testopic] 
[o.a.f.s.k.KafkaConsumerWorker.run:329] - testgroup:testopic:: exception 
occurred in kafka source worker. backing off for 1000 millis
org.apache.kafka.clients.consumer.NoOffsetForPartitionException: Undefined 
offset with no reset policy for partition: testopic-0
at 
org.apache.kafka.clients.consumer.internals.Fetcher.resetOffset(Fetcher.java:375)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.internals.Fetcher.updateFetchPositions(Fetcher.java:248)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.updateFetchPositions(KafkaConsumer.java:1601)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1034)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:995) 
~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.flume.source.kafka.KafkaConsumerWorker.fetchNextBatch(KafkaConsumerWorker.java:350)
 ~[flume-kafka-source-1.6.0.47.jar:1.6.0.47]
at 
org.apache.flume.source.kafka.KafkaConsumerWorker.run(KafkaConsumerWorker.java:291)
 ~[flume-kafka-source-1.6.0.47.jar:1.6.0.47]
27 Mar 2018 23:06:43.743 ERROR [testgroup:testopic] 
[o.a.f.s.k.KafkaConsumerWorker.run:329] - testgroup:testopic:: exception 
occurred in kafka source worker. backing off for 1000 millis
org.apache.kafka.clients.consumer.NoOffsetForPartitionException: Undefined 
offset with no reset policy for partition: testopic-0
at 
org.apache.kafka.clients.consumer.internals.Fetcher.resetOffset(Fetcher.java:375)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.internals.Fetcher.resetOffsetsIfNeeded(Fetcher.java:228)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.updateFetchPositions(KafkaConsumer.java:1591)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1034)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:995) 
~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.flume.source.kafka.KafkaConsumerWorker.fetchNextBatch(KafkaConsumerWorker.java:350)
 ~[flume-kafka-source-1.6.0.47.jar:1.6.0.47]
at 
org.apache.flume.source.kafka.KafkaConsumerWorker.run(KafkaConsumerWorker.java:291)
 ~[flume-kafka-source-1.6.0.47.jar:1.6.0.47]
27 Mar 2018 23:06:44.979 ERROR [testgroup:testopic] 
[o.a.f.s.k.KafkaConsumerWorker.run:329] - testgroup:testopic:: exception 
occurred in kafka source worker. backing off for 1000 millis
org.apache.kafka.clients.consumer.NoOffsetForPartitionException: Undefined 
offset with no reset policy for partition: testopic-0
at 
org.apache.kafka.clients.consumer.internals.Fetcher.resetOffset(Fetcher.java:375)
 ~[kafka-clients-0.10.2.1.jar:na]
at 
org.apache.kafka.clients.consumer.internals.Fetcher.resetOffsetsIfNee

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

2018-03-31 Thread Ted Yu (JIRA)

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

Ted Yu commented on KAFKA-6681:
---

I assume you use StickyAssignor.

Can you switch to RoundRobinAssignor and see if the problem persists ?

Thanks

> 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)


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

2018-04-02 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com]
I assume you are asking about this config partition.assignment.strategy. For 
this, we are using the default class class 
org.apache.kafka.clients.consumer.RangeAssignor 
Will this have the issue?


> 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)


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

2018-04-02 Thread Ted Yu (JIRA)

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

Ted Yu commented on KAFKA-6681:
---

I am digging into RangeAssignor.

Meanwhile, please try out RoundRobinAssignor.

Thanks

> 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)


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

2018-04-02 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com], Thanks for looking into this problem. As you suggested, 
we will try out RoundRobinAssignor  and let you know.

> 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)


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

2018-04-02 Thread Ted Yu (JIRA)

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

Ted Yu commented on KAFKA-6681:
---

0.10.2.1 was quite old.

It would be nice if you can reproduce this behavior (in dev cluster) using 
1.1.0 release.

Thanks

> 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)


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

2018-04-02 Thread Narayan Periwal (JIRA)

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

Narayan Periwal commented on KAFKA-6681:


[~yuzhih...@gmail.com], I understand your concern. 0.10.2.1 is a bit old.
The thing is we have not been able to reproduce this in our dev environment 
even with 0.10.2.1. We have hit this issue only in production 3 times.
If we are able to find a way to reproduce this in 0.10.2.1, then definitely we 
can give a try on 1.1.0. What do you suggest?

> 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)


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

2018-06-25 Thread Steven Aerts (JIRA)


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

Steven Aerts commented on KAFKA-6681:
-

I think this problem is a duplicate of [KAFKA-7026].

> 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)


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

2018-06-25 Thread Narayan Periwal (JIRA)


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

Narayan Periwal commented on KAFKA-6681:


[~steven.aerts], We are using RangeAssignor(which is the default), and not the 
Sticky Assignor which KAFKA-7026 mentions of.

> 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)