[ https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14963591#comment-14963591 ]
Jun Rao commented on KAFKA-2017: -------------------------------- Another option that we have in the 0.9.0 release is not to persist the consumer state at all, but simply relax the checking during offset commit. So, we can allow any consumer to commit the offsets, regardless of its consumer id and generation id. The current checking during offset commit really helps when there is soft failure in the consumer (e.g., due to GC), which should be rare in the new consumer given the high default session timeout. So, in the normal case, I am not sure if this checking buys us much. The main benefit of this approach is that the implementation will be much easier in 0.9.0. This will give us more time to think through the story (ZK based vs Kafka based) on persisting consumer states post 0.9.0. Also, the current proposal still requires existing consumers to rebalance during coordinator failover. If we can avoid that, it would be even better. [~toddpalino], currently the offset topic is auto-created on first use. Hopefully by that time, the whole Kafka cluster is already started. > Persist Coordinator State for Coordinator Failover > -------------------------------------------------- > > Key: KAFKA-2017 > URL: https://issues.apache.org/jira/browse/KAFKA-2017 > Project: Kafka > Issue Type: Sub-task > Components: consumer > Affects Versions: 0.9.0.0 > Reporter: Onur Karaman > Assignee: Guozhang Wang > Fix For: 0.9.0.0 > > Attachments: KAFKA-2017.patch, KAFKA-2017_2015-05-20_09:13:39.patch, > KAFKA-2017_2015-05-21_19:02:47.patch > > > When a coordinator fails, the group membership protocol tries to failover to > a new coordinator without forcing all the consumers rejoin their groups. This > is possible if the coordinator persists its state so that the state can be > transferred during coordinator failover. This state consists of most of the > information in GroupRegistry and ConsumerRegistry. -- This message was sent by Atlassian JIRA (v6.3.4#6332)