Hey Onur,

Thanks for the well-written KIP! I have two questions below.

1) In the process of migrating from OZKCCs and MDZKCCs to MEZKCCs, we will
may a mix of OZKCCs, MDZKCCs and MEZKCCs. OZKCC and MDZKCC will only commit
to zookeeper and MDZKCC will use kafka-based offset storage. Would we lose
offset committed to zookeeper by a MDZKCC if a partition ownership if
transferred from a MDZKCC to a MEZKCC?

2) Suppose every process in the group is running MEZKCC. Each MEZKCC has a
zookeeper-based partition assignment and kafka-based partition assignment.
Is it guaranteed that these two assignments are exactly the same across
processes? If not, say the zookeeper-based assignment assigns p1, p2 to
process 1, and p3 to process 2. And kafka-based assignment assigns p1, p3
to process 1, and p2 to process 2. Say process 1 handles receives the
notification to switch to kafka-based notification before process 2, it is
possible that during a short period of time p3 will be consumed by both
processes?

This period is probably short and I am not sure how many messages may be
duplicated as a result. But it seems possible to avoid this completely
according to an idea that Becket suggested in a previous discussion. The
znode /consumers/<group id>/migration/mode can contain a sequence number
that increment for each switch. Say the znode is toggled to kafka with
sequence number 2, each MEZKCC will commit offset to with number 2 in the
metadata for partitions that it currently owns according to the zk-based
partition assignment, and then periodically fetches the committed offset
and the metadata for the partitions that it should own according to the
kafka-based partition assignment. Each MEZKCC only starts consumption when
the metadata has incremented to the number 2.

Thanks,
Dong








On Mon, Feb 20, 2017 at 12:04 PM, Onur Karaman <onurkaraman.apa...@gmail.com
> wrote:

> Hey everyone.
>
> I made a KIP that provides a mechanism for migrating from
> ZookeeperConsumerConnector to KafkaConsumer as well as a mechanism for
> rolling back from KafkaConsumer to ZookeeperConsumerConnector:
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-125%
> 3A+ZookeeperConsumerConnector+to+KafkaConsumer+Migration+and+Rollback
>
> Comments are welcome.
>
> - Onur
>

Reply via email to