I like the idea too. Once all the old consumers have upgraded to this new embedded KafkaConsumer and they switch over to the new consumer’s group assignment protocol, does anything change when a real KafkaConsumer joins the group? At this point, if we have say N consumers in the group, we will have N-1 old consumers with embedded KafkaConsumer and 1 new KafkaConsumer. Are all the KafkaConsumers (embedded and non-embedded) “equal” in terms of how group assignment works?
I recall that in group assignment, one of the consumers becomes the coordinator of the group and sends assignments to the others (do I have that right?) Can the embedded KafkaConsumers act as the coordinator for the group as well? How much would we have to expose the embedded KafkaConsumer? For example, would we add bootstrap.servers as a config to the old consumer? Probably not, since it can be derived from zookeeper.connect? There are various configs in the old consumer related to rebalancing, like rebalance.max.retries, rebalance.backoff.ms. Do those get ignored when using the embedded KafkaConsumer? If so, then developers will probably need to be aware of how the behavior changes once they switch to the embedded KafkaConsumer. -James > On Nov 27, 2016, at 7:07 PM, Jason Gustafson <ja...@confluent.io> wrote: > > Onur's suggestion or something like it sounds like it could work. Suppose > we add some metadata in Zookeeper for consumers which support the embedded > KafkaConsumer. Until all members in the group have declared support, the > consumer will continue use Zk for their partition assignments. But once all > members support the embedded consumer, then they will switch to receiving > their assignments from the embedded KafkaConsumer. So basically upgrading > to the new consumer first requires that you upgrade the old consumer to use > the new consumer's group assignment protocol. Once you've done that, then > upgrading to the new consumer becomes straightforward. Does that work? Then > maybe you don't need to propagate any extra information over the rebalance > protocol. > > -Jason > > On Wed, Nov 23, 2016 at 12:35 AM, Onur Karaman <onurkaraman.apa...@gmail.com >> wrote: > >> Some coworkers may have had issues seeing my earlier post so reposting >> under a different email: >> >> So my earlier stated suboptimal migration plans and Joel's idea all suffer >> from either downtime or dual partition ownership and consumption. >> >> But I think there's a bigger problem: they assume users are willing to do >> the full migration immediately. I'm not convinced that this is realistic. >> Some teams may be okay with this (and the earlier stated consequences of >> the existing approaches), but others want to "canary" new code. That is, >> they want to deploy a single instance of the new code to test the waters >> while all the other instances run old code. It's not unreasonable for this >> to span days. In this world, earlier alternatives would have the canary >> under heavy load since it is the sole new consumer in the group and it is >> guaranteed to own every partition the group is interested in. So the canary >> is likely going to look unhealthy and the consumer can fall behind. >> >> Here's a not-fully-thought-out idea: >> Suppose we roll out a ZookeeperConsumerConnector that uses an embedded >> KafkaConsumer to passively participate in kafka-based coordination while >> still participating in zookeeper-based coordination. For now, the >> ZookeeperConsumerConnectors just uses the partition assignment as decided >> in zookeeper. Now suppose an outside KafkaConsumer comes up. Kafka-based >> coordination allows arbitrary metadata to get broadcasted to the group. >> Maybe we can somehow broadcast a flag saying a new consumer is running >> during this migration. If the KafkaConsumers embedded in the >> ZookeeperConsumerConnector see this flag, then they can notify the >> ZookeeperConsumerConnector's fetchers to fetch the partitions defined by >> the kafka-based coordination rebalance result. The >> ZookeeperConsumerConnector's embedded KafkaConsumer's fetchers never get >> used at any point in time. >> >> The benefits of this approach would be: >> 1. no downtime >> 2. minimal window of dual partition ownership >> 3. even partition distribution upon canary arrival. >> ZookeeperConsumerConnector instances can claim some partition ownership, so >> the canaried KafkaConsumer doesn't get overwhelmed by all of the >> partitions. >> >> On Fri, Nov 18, 2016 at 12:54 PM, Onur Karaman < >> okara...@linkedin.com.invalid> wrote: >> >>> So my earlier stated suboptimal migration plans and Joel's idea all >> suffer >>> from either downtime or dual partition ownership and consumption. >>> >>> But I think there's a bigger problem: they assume users are willing to do >>> the full migration immediately. I'm not convinced that this is realistic. >>> Some teams may be okay with this (and the earlier stated consequences of >>> the existing approaches), but others want to "canary" new code. That is, >>> they want to deploy a single instance of the new code to test the waters >>> while all the other instances run old code. It's not unreasonable for >> this >>> to span days. In this world, earlier alternatives would have the canary >>> under heavy load since it is the sole new consumer in the group and it is >>> guaranteed to own every partition the group is interested in. So the >> canary >>> is likely going to look unhealthy and the consumer can fall behind. >>> >>> Here's a not-fully-thought-out idea: >>> Suppose we roll out a ZookeeperConsumerConnector that uses an embedded >>> KafkaConsumer to passively participate in kafka-based coordination while >>> still participating in zookeeper-based coordination. For now, the >>> ZookeeperConsumerConnectors just uses the partition assignment as decided >>> in zookeeper. Now suppose an outside KafkaConsumer comes up. Kafka-based >>> coordination allows arbitrary metadata to get broadcasted to the group. >>> Maybe we can somehow broadcast a flag saying a new consumer is running >>> during this migration. If the KafkaConsumers embedded in the >>> ZookeeperConsumerConnector see this flag, then they can notify the >>> ZookeeperConsumerConnector's fetchers to fetch the partitions defined by >>> the kafka-based coordination rebalance result. The >>> ZookeeperConsumerConnector's embedded KafkaConsumer's fetchers never get >>> used at any point in time. >>> >>> The benefits of this approach would be: >>> 1. no downtime >>> 2. minimal window of dual partition ownership >>> 3. even partition distribution upon canary arrival. >>> ZookeeperConsumerConnector instances can claim some partition ownership, >> so >>> the canaried KafkaConsumer doesn't get overwhelmed by all of the >>> partitions. >>> >>> On Thu, Nov 17, 2016 at 9:17 PM, Joel Koshy <jjkosh...@gmail.com> wrote: >>> >>>> Not sure it is worth doing, but a simple migration approach that avoids >>>> *service* downtime could be as follows: >>>> >>>> - Add a “migration mode” to the old consumer that disables its >>> fetchers >>>> and disables offset commits. i.e., the consumers rebalance and own >>>> partitions but do basically nothing. >>>> - So assuming the old consumer is already committing offsets to >> Kafka, >>>> the process would be: >>>> - Bounce the consumer group (still on the old consumer) with: >>>> - Migration mode on >>>> - consumer.timeout.ms -1 >>>> - Bounce the consumer group to switch to the new consumer >>>> - i.e., effectively pause and resume the entire group without real >>>> downtime of the services. >>>> >>>> >>>> >>>> On Thu, Nov 17, 2016 at 7:30 PM, Ismael Juma <ism...@juma.me.uk> >> wrote: >>>> >>>>> Thanks James. I had read your post and was planning to find it in >> order >>>> to >>>>> share it here so you saved me some work. :) >>>>> >>>>> Ismael >>>>> >>>>> On Fri, Nov 18, 2016 at 3:21 AM, James Cheng <wushuja...@gmail.com> >>>> wrote: >>>>> >>>>>> Sorry to self-plug, but I wrote a blog post that talks about this, >>> with >>>>>> respect to mirrormaker. I came to the same 3 solutions that Onur >>>>> described. >>>>>> >>>>>> https://logallthethings.com/2016/10/07/mirrormaker- >>>>>> gotchas-when-moving-from-the-old-consumer-to-the-new-consumer/ < >>>>>> https://logallthethings.com/2016/10/07/mirrormaker- >>>>>> gotchas-when-moving-from-the-old-consumer-to-the-new-consumer/> >>>>>> >>>>>> -James >>>>>> >>>>>>> On Nov 17, 2016, at 7:37 AM, Ismael Juma <ism...@juma.me.uk> >>> wrote: >>>>>>> >>>>>>> Hi Onur, >>>>>>> >>>>>>> It is a good point that there is currently no out of the box >>> solution >>>>> for >>>>>>> migrating from the old consumer to the new consumer where neither >>>>>> downtime >>>>>>> or duplicate consumption are acceptable. As I understand, this is >>>>>> important >>>>>>> for some of the usages at LinkedIn. Do you have any plans to >> tackle >>>>> this >>>>>>> issue? >>>>>>> >>>>>>> Jason, any thoughts on this? >>>>>>> >>>>>>> Ismael >>>>>>> >>>>>>> On Mon, Oct 31, 2016 at 11:03 PM, Onur Karaman < >>>>>>> okara...@linkedin.com.invalid> wrote: >>>>>>> >>>>>>>> Does this make sense given that we still don't have a graceful >>>>> migration >>>>>>>> plan from the old to new consumer? >>>>>>>> >>>>>>>> Different suboptimal migration plans that I can think of are: >>>>>>>> 1. shutdown all the old consumers of a group first and start >> them >>>> back >>>>>> up >>>>>>>> with the new consumer, causing downtime. >>>>>>>> 2. have a mix of old and new consumers in the same group, >> causing >>>>>> duplicate >>>>>>>> partition ownership and consumption as each rebalance protocol >>>> ignores >>>>>> the >>>>>>>> other. >>>>>>>> 3. form a brand new group for the new consumers doing the same >>> work >>>> as >>>>>> the >>>>>>>> old consumer group, still causing duplicate partition ownership >>> and >>>>>>>> consumption across the two groups. >>>>>>>> >>>>>>>> On Mon, Oct 31, 2016 at 3:42 PM, Jun Rao <j...@confluent.io> >>> wrote: >>>>>>>> >>>>>>>>> Starting to deprecate the old consumer in the next release >> seems >>>>> like a >>>>>>>>> good idea. >>>>>>>>> >>>>>>>>> Thanks, >>>>>>>>> >>>>>>>>> Jun >>>>>>>>> >>>>>>>>> On Tue, Oct 25, 2016 at 2:45 AM, Ismael Juma < >> ism...@juma.me.uk> >>>>>> wrote: >>>>>>>>> >>>>>>>>>> Hi all, >>>>>>>>>> >>>>>>>>>> In 0.10.1.0, we removed the beta label from the new Java >>> consumer >>>>>>>>>> documentation and updated the various tools so that they can >> use >>>> the >>>>>>>> new >>>>>>>>>> consumer without having to pass the `--new-consumer` flag >> (more >>>>>>>>>> specifically the new consumer is used if `bootstrap-server` is >>>> set). >>>>>>>> More >>>>>>>>>> details of the reasoning can be found in the original discuss >>>>> thread: >>>>>>>>>> http://search-hadoop.com/m/Kafka/uyzND1e4bUP1Rjq721 >>>>>>>>>> >>>>>>>>>> The old consumers don't have security or `offsetsForTimestamp` >>>>>> (KIP-79) >>>>>>>>>> support and the plan is to only add features to the new Java >>>>> consumer. >>>>>>>>> Even >>>>>>>>>> so, the old consumers are a significant maintenance burden as >>> they >>>>>>>>>> duplicate protocol request/response classes (the >> SimpleConsumer >>>>>> exposes >>>>>>>>>> them in the public API sadly). I experienced this first hand >>> most >>>>>>>>> recently >>>>>>>>>> while working on KIP-74. >>>>>>>>>> >>>>>>>>>> Given the above, I propose we deprecate the old consumers in >>> trunk >>>>> to >>>>>>>>> nudge >>>>>>>>>> users in the right direction. Users will have the 0.10.1.0 >> cycle >>>> to >>>>>>>> start >>>>>>>>>> migrating to the new Java consumer with no build warnings. >> Once >>>> they >>>>>>>>>> upgrade to the next release (i.e. 0.10.2.0), users who are >> still >>>>> using >>>>>>>>> the >>>>>>>>>> old consumers will get warnings at build time encouraging them >>> to >>>>> move >>>>>>>> to >>>>>>>>>> the new consumer, but everything will still work fine. >>>>>>>>>> >>>>>>>>>> In a future major release, the old consumers (along with the >> old >>>>>>>>> producers) >>>>>>>>>> will be removed. We will have a separate discuss/vote thread >> for >>>>> that >>>>>>>> to >>>>>>>>>> make sure the time is right. >>>>>>>>>> >>>>>>>>>> Thoughts? >>>>>>>>>> >>>>>>>>>> Ismael >>>>>>>>>> >>>>>>>>> >>>>>>>> >>>>>> >>>>>> >>>>> >>>> >>> >>