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

Narayan Periwal commented on KAFKA-7026:
----------------------------------------

[~vahid], Agree that this may not be the actual cause of the issue. But we have 
seen this occuring  multiple times in our production setup, with consumers 
continuing to consume the same partition unless a manual restart is triggered. 
So, it could be due to some other issue.

[~steven.aerts], we are using custom checkpointing in zookeeper, so the 
kafka-consumer-groups.sh script to describe the consumer group does not work 
for us. 
However, we have a mechanism to detect multiple consumers when consuming from 
the same partition. I am sharing the distribution of one such case. Topic - 
test, consumer group - group1, consumers - c1,c2,c3,c4,c5.
Partition 3,4,5 of this topic were being consumed by multiple consumer 
instances.
{noformat}
group: group1, topic: test, partition: 0, consumer: c2
group: group1, topic: test, partition: 1, consumer: c4
group: group1, topic: test, partition: 2, consumer: c4
group: group1, topic: test, partition: 3, consumer: c3,c4
group: group1, topic: test, partition: 4, consumer: c3,c5
group: group1, topic: test, partition: 5, consumer: c3,c5
group: group1, topic: test, partition: 6, consumer: c5
group: group1, topic: test, partition: 7, consumer: c1
group: group1, topic: test, partition: 8, consumer: c1
group: group1, topic: test, partition: 9, consumer: c1
{noformat} 

> Sticky assignor could assign a partition to multiple consumers
> --------------------------------------------------------------
>
>                 Key: KAFKA-7026
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7026
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients
>            Reporter: Vahid Hashemian
>            Assignee: Vahid Hashemian
>            Priority: Major
>             Fix For: 2.1.0
>
>
> In the following scenario sticky assignor assigns a topic partition to two 
> consumers in the group:
>  # Create a topic {{test}} with a single partition
>  # Start consumer {{c1}} in group {{sticky-group}} ({{c1}} becomes group 
> leader and gets {{test-0}})
>  # Start consumer {{c2}}  in group {{sticky-group}} ({{c1}} holds onto 
> {{test-0}}, {{c2}} does not get any partition) 
>  # Pause {{c1}} (e.g. using Java debugger) ({{c2}} becomes leader and takes 
> over {{test-0}}, {{c1}} leaves the group)
>  # Resume {{c1}}
> At this point both {{c1}} and {{c2}} will have {{test-0}} assigned to them.
>  
> The reason is {{c1}} still has kept its previous assignment ({{test-0}}) from 
> the last assignment it received from the leader (itself) and did not get the 
> next round of assignments (when {{c2}} became leader) because it was paused. 
> Both {{c1}} and {{c2}} enter the rebalance supplying {{test-0}} as their 
> existing assignment. The sticky assignor code does not currently check and 
> avoid this duplication.
>  
> Note: This issue was originally reported on 
> [StackOverflow|https://stackoverflow.com/questions/50761842/kafka-stickyassignor-breaking-delivery-to-single-consumer-in-the-group].



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

Reply via email to