I forgot to mention that in that above idea, the "coordination.migration.mode" config would default to "auto".
On Fri, Mar 10, 2017 at 1:08 AM, Onur Karaman <onurkaraman.apa...@gmail.com> wrote: > Hey Jason. > > Thanks for the comments! > > Regarding 1: I've been considering something like this for a while now. > KIP-122 has a similar issue and I brought up some hacks in that discussion > to work around it (http://markmail.org/message/kk4ng74riejidify). While > solving this problem would help loosen the requirements for migration, it > seems beyond the scope of this KIP. It's hard to say whether we should be > trying to solve that issue here. > > Regarding 2: I agree that we should offer a tool somewhere to help with > the migration and do the toggle. It's not clear to me if we should put it > in kafka-consumer-groups.sh or in some new migration script. > > Regarding general migration complexity: something else Joel and I had > considered was the ability to optionally create the toggle on startup to > skip the step of having to manually set the toggle. There are many ways we > can do this. > > As an example, we can rid of the notion of "coordination.migration.enabled" > and just have a config called "coordination.migration.mode" whose values > can be {"off", "manual", "auto"} where: > > - "off" would act like "coordination.migration.enabled" set to false. > We do not participate in coordination migration. > - "manual" would act like "coordination.migration.enabled" set to true > in the current KIP proposal. Do not attempt to create the toggle on > startup, but spin up an EKC and be ready to react to the toggle. This mode > helps an org gradually migrate to or rollback from kafka-based > coordination. > - "auto" would act like "coordination.migration.enabled" set to true > in the current KIP proposal but additionally attempt to create the toggle > with "kafka" on startup if the znode doesn't already exist. The same rules > from the KIP apply where if a OZKCC or MDZKCC exists, the value is ignored > and we just use zookeeper-based coordination. This mode lets us skip the > step of having to manually set the toggle. > > Let me know what you think! > > On Thu, Mar 9, 2017 at 10:30 AM, Jason Gustafson <ja...@confluent.io> > wrote: > >> Hey Onur, >> >> Sorry for the late reply. Thanks for the well-written KIP! I think the >> proposal makes sense. The only thing I was wondering is whether the >> process >> is a bit complex for most users. You'd probably have no trouble at LI >> (especially given you're implementing it!), but I'm not so sure about the >> users who aren't as close to the Kafka internals. That said, I don't see >> any great options to simplify the process, and having this approach is >> better than having none, so maybe it's fine. Here are a couple minor >> suggestions: >> >> 1. One thought that came to mind is whether it would be worthwhile to add >> a >> broker config to disable the group membership check for offset commits. >> This would simplify the process by eliminating the initial step of turning >> off offset commits in Kafka for the group to be migrated prior to turning >> on group coordination through Kafka. I'm not thrilled about this option >> since it removes the protection that that check provides (I guess this is >> no worse than using Kafka for offsets storage with the old consumer >> anyway). Also it's a config we'd to ultimately have to deprecate and >> remove. >> >> 2. It seems like the toggle on the group's coordination mode is done >> manually. Should we add that to consumer-groups.sh? >> >> Thanks, >> Jason >> >> On Thu, Feb 23, 2017 at 1:22 PM, Dong Lin <lindon...@gmail.com> wrote: >> >> > Yeah, I agree it is a bit complex to do that approach for a one-time >> > migration. Probably not worth it. Here is another idea to reduce, but >> not >> > eliminate, the amount of message duplication during migration. I am fine >> > with not doing it. Just want to see the opinion from open source >> community. >> > >> > The problem with current solution is that, when we toggle the zookeeper >> > path in order to migrate from MEZKCC, with 50% probability the old >> owner of >> > the partition may reduce notification later than the new partition >> owner. >> > Thus the new partition owner may reduce the offset committed by the >> older >> > owner 5 sec ago assuming the auto-commit interval is 10 sec. The >> messages >> > produced in this 5 sec window may be consumed multiple times. This >> amount >> > is even more if consumer is bootstrapping. >> > >> > One way to mitigate this problem is for the MEZKCC to sleep for a >> > configurable amount of time after it receives zookeeper notification but >> > before it starts to fetch offset and consume message. This seems like an >> > easy change that allows user to tradeoff between the message duplication >> > and consumer downtime. >> > >> > >> > >> > On Thu, Feb 23, 2017 at 11:20 AM, Joel Koshy <jjkosh...@gmail.com> >> wrote: >> > >> > > Regarding (2) - yes that's a good point. @Onur - I think the KIP >> should >> > > explicitly call this out. >> > > It is something that we did consider and decided against optimizing >> for. >> > > i.e., we just wrote that off as a minor caveat of the upgrade path in >> > that >> > > there will be a few duplicates, but not too many given that we expect >> the >> > > period of duplicate ownership to be minimal. Although it could be >> > addressed >> > > as you described, it does add complexity to an already-rather-complex >> > > migration path. Given that it is a transition state (i.e., migration) >> we >> > > felt it would be better and sufficient to keep it only as complex as >> it >> > > needs to be. >> > > >> > > On Mon, Feb 20, 2017 at 4:45 PM, Onur Karaman < >> > > onurkaraman.apa...@gmail.com> >> > > wrote: >> > > >> > > > Regarding 1: We won't lose the offset from zookeeper upon partition >> > > > transfer from OZKCC/MDZKCC to MEZKCC because MEZKCC has >> > > > "dual.commit.enabled" set to true as well as "offsets.storage" set >> to >> > > > kafka. The combination of these configs results in the consumer >> > fetching >> > > > offsets from both kafka and zookeeper and just picking the greater >> of >> > the >> > > > two. >> > > > >> > > > On Mon, Feb 20, 2017 at 4:33 PM, Dong Lin <lindon...@gmail.com> >> wrote: >> > > > >> > > > > 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 >> > > > > > >> > > > > >> > > > >> > > >> > >> > >