[
https://issues.apache.org/jira/browse/KAFKA-2017?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14957348#comment-14957348
]
Joel Koshy commented on KAFKA-2017:
-----------------------------------
If we reused this topic, then I think we could end up doing something like this:
* Bump up the key format version and add a new field: key-type (which could be
either _offset_ or _consumer group data_)
* The actual key-bytes would be either the group-topic-partition schema (for
keys with type _offset_) and just group (for keys with type _consumer group
data_)
* If the key is of type _offset_ then the value will be the same as what we
currently have for offsets
* If the key is of type _consumer group data_ then the value will be a new
schema (for the consumer group data). Now we may end up needing to support
different value schemas for different group management use-cases.
Overall, it can get a bit ugly so I'm wondering if the above indicates that it
is cleaner to just use a separate topic for group state and refactor the offset
manager to become a simple state store that can be easily repurposed for these
use-cases (offset state storage and group state storage).
> 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)