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

Narayan Periwal edited comment on KAFKA-7026 at 6/27/18 4:11 AM:
-----------------------------------------------------------------

[~vahid], unfortunately we are not able to reproduce this in our QA setup. Only 
co-relation that we have seen is this seems to happen when there is spike in 
the number of under replicated partitions in the kafka cluster. 

One more thing is when this issue happens, we have seen our consumers not 
processing data for more than "max.poll.interval.ms", thus the consumer.poll() 
call is not invoked for "max.poll.interval.ms", which means the consumer is 
considered failed and the group will rebalance in order to reassign the 
partitions to another member. Looks like, the first consumer, after 
recovery(able to process now), is still getting data from the earlier assigned 
partition, leading to this issue.


was (Author: nperiwal):
[~vahid], unfortunately we are not able to reproduce this in our QA setup. Only 
co-relation that we have seen is this seems to happen when there is spike in 
the number of under replicated partitions in the kafka cluster. 

One more thing is when this issue happens, we have seen our consumers not 
processing data for more than "max.poll.interval.ms", thus the consumer.poll() 
call is not invoked for "max.poll.interval.ms", which means the consumer is 
considered failed and the group will rebalance in order to reassign the 
partitions to another member. Looks like, the old consumer, after recovery, is 
still getting data from the earlier assigned partition, leading to this issue.

> 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