[ https://issues.apache.org/jira/browse/KAFKA-10284?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17160216#comment-17160216 ]
Boyang Chen commented on KAFKA-10284: ------------------------------------- If we have a combined scenario like below: # A static member X joins the group and updates member.id to M1, then gets stuck # Another static member Y with the same instance.id joins and updates member.id to M2, while starts working and commit offsets # The group coordinator migrates, and the member.id for the same static member rewinds to M1 # The static member X goes back online, and validated. It would try to fetch from Y's committed offset In this flow, I don't think we are violating the offset committing policy here. The only downside I could think of is that there is only one member Y who will get fenced by itself after the immigration as stated in the KIP. [~guozhang] > Group membership update due to static member rejoin should be persisted > ----------------------------------------------------------------------- > > Key: KAFKA-10284 > URL: https://issues.apache.org/jira/browse/KAFKA-10284 > Project: Kafka > Issue Type: Bug > Components: consumer > Affects Versions: 2.3.0, 2.4.0, 2.5.0, 2.6.0 > Reporter: Boyang Chen > Assignee: Boyang Chen > Priority: Major > Fix For: 2.6.1 > > > For known static members rejoin, we would update its corresponding member.id > without triggering a new rebalance. This serves the purpose for avoiding > unnecessary rebalance for static membership, as well as fencing purpose if > some still uses the old member.id. > The bug is that we don't actually persist the membership update, so if no > upcoming rebalance gets triggered, this new member.id information will get > lost during group coordinator immigration, thus bringing up the zombie member > identity. > The bug find credit goes to [~hachikuji] -- This message was sent by Atlassian Jira (v8.3.4#803005)