You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by Mike Freyberger <mi...@xandr.com> on 2018/11/04 22:13:23 UTC

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Boyang,

Thanks for updating the KIP. It's shaping up well. Two things:

1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do not think a bounce of the leader should trigger a rebalance.

2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on shrink down. What do you think?

Mike

On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:

    Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look:
    
    https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
    
    KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
    cwiki.apache.org
    For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
    
    
    
    
    
    ________________________________
    From: Boyang Chen <bc...@outlook.com>
    Sent: Monday, October 29, 2018 12:34 PM
    To: dev@kafka.apache.org
    Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
    
    Thanks everyone for the input on this thread! (Sorry it's been a while) I feel that we are very close to the final solution.
    
    
    Hey Jason and Mike, I have two quick questions on the new features here:
    
      1.  so our proposal is that until we add a new static member into the group (scale up), we will not trigger rebalance until the "registration timeout"( the member has been offline for too long)? How about leader's rejoin request, I think we should still trigger rebalance when that happens, since the consumer group may have new topics to consume?
      2.  I'm not very clear on the scale up scenario in static membership here. Should we fallback to dynamic membership while adding/removing hosts (by setting member.name = null), or we still want to add instances with `member.name` so that we eventually expand/shrink the static membership? I personally feel the easier solution is to spin up new members and wait until either the same "registration timeout" or a "scale up timeout" before starting the rebalance. What do you think?
    
    Meanwhile I will go ahead to make changes to the KIP with our newly discussed items and details. Really excited to see the design has become more solid.
    
    Best,
    Boyang
    
    ________________________________
    From: Jason Gustafson <ja...@confluent.io>
    Sent: Saturday, August 25, 2018 6:04 AM
    To: dev
    Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
    
    Hey Mike,
    
    Yeah, that's a good point. A long "registration timeout" may not be a great
    idea. Perhaps in practice you'd set it long enough to be able to detect a
    failure and provision a new instance. Maybe on the order of 10 minutes is
    more reasonable.
    
    In any case, it's probably a good idea to have an administrative way to
    force deregistration. One option is to extend the DeleteGroups API with a
    list of members names.
    
    -Jason
    
    
    
    On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <mf...@appnexus.com>
    wrote:
    
    > Jason,
    >
    > Regarding step 4 in your proposal which suggests beginning a long timer
    > (30 minutes) when a static member leaves the group, would there also be the
    > ability for an admin to force a static membership expiration?
    >
    > I’m thinking that during particular types of outages or upgrades users
    > would want forcefully remove a static member from the group.
    >
    > So the user would shut the consumer down normally, which wouldn’t trigger
    > a rebalance. Then the user could use an admin CLI tool to force remove that
    > consumer from the group, so the TopicPartitions that were previously owned
    > by that consumer can be released.
    >
    > At a high level, we need consumer groups to gracefully handle intermittent
    > failures and permanent failures. Currently, the consumer group protocol
    > handles permanent failures well, but does not handle intermittent failures
    > well (it creates unnecessary rebalances). I want to make sure the overall
    > solution here handles both intermittent failures and permanent failures,
    > rather than sacrificing support for permanent failures in order to provide
    > support for intermittent failures.
    >
    > Mike
    >
    > Sent from my iPhone
    >
    > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io> wrote:
    > >
    > > Hey Guozhang,
    > >
    > > Responses below:
    > >
    > > Originally I was trying to kill more birds with one stone with KIP-345,
    > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
    > >> multi-instance client (mentioned as case 1)/2) in my early email), and
    > >> hence proposing to have a pure static-membership protocol. But thinking
    > >> twice about it I now feel it may be too ambitious and worth fixing in
    > >> another KIP.
    > >
    > >
    > > I was considering an extension to support pre-initialization of the
    > static
    > > members of the group, but I agree we should probably leave this problem
    > for
    > > future work.
    > >
    > > 1. How this longish static member expiration timeout defined? Is it via a
    > >> broker, hence global config, or via a client config which can be
    > >> communicated to broker via JoinGroupRequest?
    > >
    > >
    > > I am not too sure. I tend to lean toward server-side configs because they
    > > are easier to evolve. If we have to add something to the protocol, then
    > > we'll be stuck with it forever.
    > >
    > > 2. Assuming that for static members, LEAVE_GROUP request will not
    > trigger a
    > >> rebalance immediately either, similar to session timeout, but only the
    > >> longer member expiration timeout, can we remove the internal "
    > >> internal.leave.group.on.close" config, which is a quick walk-around
    > then?
    > >
    > >
    > > Yeah, I hope we can ultimately get rid of it, but we may need it for
    > > compatibility with older brokers. A related question is what should be
    > the
    > > behavior of the consumer if `member.name` is provided but the broker
    > does
    > > not support it? We could either fail or silently downgrade to dynamic
    > > membership.
    > >
    > > -Jason
    > >
    > >
    > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <wa...@gmail.com>
    > wrote:
    > >>
    > >> Hey Jason,
    > >>
    > >> I like your idea to simplify the upgrade protocol to allow co-exist of
    > >> static and dynamic members. Admittedly it may make the coordinator-side
    > >> logic a bit more complex, but I think it worth doing it.
    > >>
    > >> Originally I was trying to kill more birds with one stone with KIP-345,
    > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
    > >> multi-instance client (mentioned as case 1)/2) in my early email), and
    > >> hence proposing to have a pure static-membership protocol. But thinking
    > >> twice about it I now feel it may be too ambitious and worth fixing in
    > >> another KIP. With that, I think what you've proposed here is a good way
    > to
    > >> go for KIP-345 itself.
    > >>
    > >> Note there are a few details in your proposal we'd still need to figure
    > >> out:
    > >>
    > >> 1. How this longish static member expiration timeout defined? Is it via
    > a
    > >> broker, hence global config, or via a client config which can be
    > >> communicated to broker via JoinGroupRequest?
    > >>
    > >> 2. Assuming that for static members, LEAVE_GROUP request will not
    > trigger a
    > >> rebalance immediately either, similar to session timeout, but only the
    > >> longer member expiration timeout, can we remove the internal "
    > >> internal.leave.group.on.close" config, which is a quick walk-around
    > then?
    > >>
    > >>
    > >>
    > >> Guozhang
    > >>
    > >>
    > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <ja...@confluent.io>
    > >> wrote:
    > >>
    > >>> Hey All,
    > >>>
    > >>> Nice to see some solid progress on this. It sounds like one of the
    > >>> complications is allowing static and dynamic registration to coexist.
    > I'm
    > >>> wondering if we can do something like the following:
    > >>>
    > >>> 1. Statically registered members (those joining the group with a
    > >> non-null `
    > >>> member.name`) maintain a session with the coordinator just like
    > dynamic
    > >>> members.
    > >>> 2. If a session is active for a static member when a rebalance begins,
    > >> then
    > >>> basically we'll keep the current behavior. The rebalance will await the
    > >>> static member joining the group.
    > >>> 3. If a static member does not have an active session, then the
    > >> coordinator
    > >>> will not wait for it to join, but will still include it in the
    > rebalance.
    > >>> The coordinator will forward the cached subscription information to the
    > >>> leader and will cache the assignment after the rebalance completes.
    > (Note
    > >>> that we still have the generationId to fence offset commits from a
    > static
    > >>> zombie if the assignment changes.)
    > >>> 4. When a static member leaves the group or has its session expire, no
    > >>> rebalance is triggered. Instead, we can begin a timer to expire the
    > >> static
    > >>> registration. This would be a longish timeout (like 30 minutes say).
    > >>>
    > >>> So basically static members participate in all rebalances regardless
    > >>> whether they have an active session. In a given rebalance, some of the
    > >>> members may be static and some dynamic. The group leader can
    > >> differentiate
    > >>> the two based on the presence of the `member.name` (we have to add
    > this
    > >> to
    > >>> the JoinGroupResponse). Generally speaking, we would choose leaders
    > >>> preferentially from the active members that support the latest
    > JoinGroup
    > >>> protocol and are using static membership. If we have to choose a leader
    > >>> with an old version, however, it would see all members in the group
    > >> (static
    > >>> or dynamic) as dynamic members and perform the assignment as usual.
    > >>>
    > >>> Would that work?
    > >>>
    > >>> -Jason
    > >>>
    > >>>
    > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <wa...@gmail.com>
    > >> wrote:
    > >>>
    > >>>> Hello Boyang,
    > >>>>
    > >>>> Thanks for the updated proposal, a few questions:
    > >>>>
    > >>>> 1. Where will "change-group-timeout" be communicated to the broker?
    > >> Will
    > >>>> that be a new field in the JoinGroupRequest, or are we going to
    > >>> piggy-back
    > >>>> on the existing session-timeout field (assuming that the original
    > value
    > >>>> will not be used anywhere in the static membership any more)?
    > >>>>
    > >>>> 2. "However, if the consumer takes longer than session timeout to
    > >> return,
    > >>>> we shall still trigger rebalance but it could still try to catch
    > >>>> `change-group-timeout`.": what does this mean? I thought your proposal
    > >> is
    > >>>> that for static memberships, the broker will NOT trigger rebalance
    > even
    > >>>> after session-timeout has been detected, but only that after
    > >>>> change-group-timeout
    > >>>> which is supposed to be longer than session-timeout to be defined?
    > >>>>
    > >>>> 3. "A join group request with member.name set will be treated as
    > >>>> `static-membership` strategy", in this case, how would the switch from
    > >>>> dynamic to static happen, since whoever changed the member.name to
    > >>>> not-null
    > >>>> will be rejected, right?
    > >>>>
    > >>>> 4. "just erase the cached mapping, and wait for session timeout to
    > >>> trigger
    > >>>> rebalance should be sufficient." this is also a bit unclear to me: who
    > >>> will
    > >>>> erase the cached mapping? Since it is on the broker-side I assume that
    > >>>> broker has to do it. Are you suggesting to use a new request for it?
    > >>>>
    > >>>> 5. "Halfway switch": following 3) above, if your proposal is basically
    > >> to
    > >>>> let "first join-request wins", and the strategy will stay as is until
    > >> all
    > >>>> members are gone, then this will also not happen since whoever used
    > >>>> different strategy as the first guy who sends join-group request will
    > >> be
    > >>>> rejected right?
    > >>>>
    > >>>>
    > >>>> Guozhang
    > >>>>
    > >>>>
    > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <jo...@confluent.io>
    > >> wrote:
    > >>>>
    > >>>>> This sounds good to me!
    > >>>>>
    > >>>>> Thanks for the time you've spent on it,
    > >>>>> -John
    > >>>>>
    > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <bc...@outlook.com>
    > >>>> wrote:
    > >>>>>
    > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
    > >> haven't
    > >>>> got
    > >>>>>> time to update this thread. To summarize what we come up so far,
    > >> here
    > >>>> is
    > >>>>> a
    > >>>>>> draft updated plan:
    > >>>>>>
    > >>>>>>
    > >>>>>> Introduce a new config called `member.name` which is supposed to
    > >> be
    > >>>>>> provided uniquely by the consumer client. The broker will maintain
    > >> a
    > >>>>> cache
    > >>>>>> with [key:member.name, value:member.id]. A join group request with
    > >>>>>> member.name set will be treated as `static-membership` strategy,
    > >> and
    > >>>>> will
    > >>>>>> reject any join group request without member.name. So this
    > >>>> coordination
    > >>>>>> change will be differentiated from the `dynamic-membership`
    > >> protocol
    > >>> we
    > >>>>>> currently have.
    > >>>>>>
    > >>>>>>
    > >>>>>> When handling static join group request:
    > >>>>>>
    > >>>>>>  1.   The broker will check the membership to see whether this is
    > >> a
    > >>>> new
    > >>>>>> member. If new, broker allocate a unique member id, cache the
    > >> mapping
    > >>>> and
    > >>>>>> move to rebalance stage.
    > >>>>>>  2.   Following 1, if this is an existing member, broker will not
    > >>>> change
    > >>>>>> group state, and return its cached member.id and current
    > >> assignment.
    > >>>>>> (unless this is leader, we shall trigger rebalance)
    > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with pair
    > >>> member
    > >>>>>> name and id, I think for join group request it is ok to leave
    > >> member
    > >>> id
    > >>>>>> blank as member name is the unique identifier. In commit offset
    > >>> request
    > >>>>> we
    > >>>>>> *must* have both.
    > >>>>>>
    > >>>>>>
    > >>>>>> When handling commit offset request, if enabled with static
    > >>> membership,
    > >>>>>> each time the commit request must have both member.name and
    > >>> member.id
    > >>>> to
    > >>>>>> be identified as a `certificated member`. If not, this means there
    > >>> are
    > >>>>>> duplicate consumer members with same member name and the request
    > >> will
    > >>>> be
    > >>>>>> rejected to guarantee consumption uniqueness.
    > >>>>>>
    > >>>>>>
    > >>>>>> When rolling restart/shutting down gracefully, the client will
    > >> send a
    > >>>>>> leave group request (static membership mode). In static membership,
    > >>> we
    > >>>>> will
    > >>>>>> also define `change-group-timeout` to hold on rebalance provided by
    > >>>>> leader.
    > >>>>>> So we will wait for all the members to rejoin the group and do
    > >>> exactly
    > >>>>> one
    > >>>>>> rebalance since all members are expected to rejoin within timeout.
    > >> If
    > >>>>>> consumer crashes, the join group request from the restarted
    > >> consumer
    > >>>> will
    > >>>>>> be recognized as an existing member and be handled as above
    > >> condition
    > >>>> 1;
    > >>>>>> However, if the consumer takes longer than session timeout to
    > >> return,
    > >>>> we
    > >>>>>> shall still trigger rebalance but it could still try to catch
    > >>>>>> `change-group-timeout`. If it failed to catch second timeout, its
    > >>>> cached
    > >>>>>> state on broker will be garbage collected and trigger a new
    > >> rebalance
    > >>>>> when
    > >>>>>> it finally joins.
    > >>>>>>
    > >>>>>>
    > >>>>>> And consider the switch between dynamic to static membership.
    > >>>>>>
    > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
    > >> membership
    > >>>> to
    > >>>>>> static and wait for all the current members to restart, since their
    > >>>>>> membership is still dynamic. Here our assumption is that the
    > >> restart
    > >>>>>> process shouldn't take a long time, as long restart is breaking the
    > >>>>>> `rebalance timeout` in whatever membership protocol we are using.
    > >>>> Before
    > >>>>>> restart, all dynamic member join requests will be rejected.
    > >>>>>>  2.  Static to dynamic: this is more like a downgrade which should
    > >>> be
    > >>>>>> smooth: just erase the cached mapping, and wait for session timeout
    > >>> to
    > >>>>>> trigger rebalance should be sufficient. (Fallback to current
    > >>> behavior)
    > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
    > >> dynamic
    > >>>>>> membership while some keep static membership. This will cause the
    > >>> group
    > >>>>>> rebalance forever without progress because dynamic/static states
    > >> are
    > >>>>>> bouncing each other. This could guarantee that we will not make the
    > >>>>>> consumer group work in a wrong state by having half static and half
    > >>>>> dynamic.
    > >>>>>>
    > >>>>>> To guarantee correctness, we will also push the member name/id pair
    > >>> to
    > >>>>>> _consumed_offsets topic (as Matthias pointed out) and upgrade the
    > >> API
    > >>>>>> version, these details will be further discussed back in the KIP.
    > >>>>>>
    > >>>>>>
    > >>>>>> Are there any concern for this high level proposal? Just want to
    > >>>>> reiterate
    > >>>>>> on the core idea of the KIP: "If the broker recognize this consumer
    > >>> as
    > >>>> an
    > >>>>>> existing member, it shouldn't trigger rebalance".
    > >>>>>>
    > >>>>>> Thanks a lot for everyone's input! I feel this proposal is much
    > >> more
    > >>>>>> robust than previous one!
    > >>>>>>
    > >>>>>>
    > >>>>>> Best,
    > >>>>>>
    > >>>>>> Boyang
    > >>>>>>
    > >>>>>> ________________________________
    > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
    > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
    > >>>>>> To: dev@kafka.apache.org
    > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
    > >>> by
    > >>>>>> specifying member id
    > >>>>>>
    > >>>>>> Hi,
    > >>>>>>
    > >>>>>> thanks for the detailed discussion. I learned a lot about internals
    > >>>> again
    > >>>>>> :)
    > >>>>>>
    > >>>>>> I like the idea or a user config `member.name` and to keep `
    > >>> member.id`
    > >>>>>> internal. Also agree with Guozhang, that reusing `client.id` might
    > >>> not
    > >>>>>> be a good idea.
    > >>>>>>
    > >>>>>> To clarify the algorithm, each time we generate a new `member.id`,
    > >>> we
    > >>>>>> also need to update the "group membership" information (ie, mapping
    > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id` replaces
    > >>> the
    > >>>>>> old entry in the cache.
    > >>>>>>
    > >>>>>> I also think, we need to preserve the `member.name -> member.id`
    > >>>> mapping
    > >>>>>> in the `__consumer_offset` topic. The KIP should mention this IMHO.
    > >>>>>>
    > >>>>>> For changing the default value of config `leave.group.on.close`. I
    > >>>> agree
    > >>>>>> with John, that we should not change the default config, because it
    > >>>>>> would impact all consumer groups with dynamic assignment. However,
    > >> I
    > >>>>>> think we can document, that if static assignment is used (ie,
    > >>>>>> `member.name` is configured) we never send a LeaveGroupRequest
    > >>>>>> regardless of the config. Note, that the config is internal, so not
    > >>>> sure
    > >>>>>> how to document this in detail. We should not expose the internal
    > >>>> config
    > >>>>>> in the docs.
    > >>>>>>
    > >>>>>> About upgrading: why do we need have two rolling bounces and encode
    > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
    > >>>>>>
    > >>>>>> If we upgrade an existing consumer group from dynamic to static, I
    > >>>> don't
    > >>>>>> see any reason why both should not work together and single rolling
    > >>>>>> bounce would not be sufficient? If we bounce the first consumer and
    > >>>>>> switch from dynamic to static, it sends a `member.name` and the
    > >>> broker
    > >>>>>> registers the [member.name, member.id] in the cache. Why would
    > >> this
    > >>>>>> interfere with all other consumer that use dynamic assignment?
    > >>>>>>
    > >>>>>> Also, Guozhang mentioned that for all other request, we need to
    > >> check
    > >>>> if
    > >>>>>> the mapping [member.name, member.id] contains the send `member.id`
    > >>> --
    > >>>> I
    > >>>>>> don't think this is necessary -- it seems to be sufficient to check
    > >>> the
    > >>>>>> `member.id` from the [member.id, Assignment] mapping as be do
    > >> today
    > >>> --
    > >>>>>> thus, checking `member.id` does not require any change IMHO.
    > >>>>>>
    > >>>>>>
    > >>>>>> -Matthias
    > >>>>>>
    > >>>>>>
    > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
    > >>>>>>> @James
    > >>>>>>>
    > >>>>>>> What you described is true: the transition from dynamic to static
    > >>>>>>> memberships are not thought through yet. But I do not think it is
    > >>> an
    > >>>>>>> impossible problem: note that we indeed moved the offset commit
    > >>> from
    > >>>> ZK
    > >>>>>> to
    > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to first to
    > >>>>>>> double-commits on both zk and coordinator, and then do a second
    > >>> round
    > >>>>> to
    > >>>>>>> turn the zk off.
    > >>>>>>>
    > >>>>>>> So just to throw a wild idea here: also following a
    > >>>> two-rolling-bounce
    > >>>>>>> manner, in the JoinGroupRequest we can set the flag to "static"
    > >>> while
    > >>>>>> keep
    > >>>>>>> the registry-id field empty still, in this case, the coordinator
    > >>>> still
    > >>>>>>> follows the logic of "dynamic", accepting the request while
    > >>> allowing
    > >>>>> the
    > >>>>>>> protocol to be set to "static"; after the first rolling bounce,
    > >> the
    > >>>>> group
    > >>>>>>> protocol is already "static", then a second rolling bounce is
    > >>>> triggered
    > >>>>>> and
    > >>>>>>> this time we set the registry-id.
    > >>>>>>>
    > >>>>>>>
    > >>>>>>> Guozhang
    > >>>>>>>
    > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
    > >> wushujames@gmail.com>
    > >>>>>> wrote:
    > >>>>>>>
    > >>>>>>>> Guozhang, in a previous message, you proposed said this:
    > >>>>>>>>
    > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <wangguoz@gmail.com
    > >>>
    > >>>>> wrote:
    > >>>>>>>>>
    > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional fields:
    > >>>>>>>>>
    > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
    > >>> protocols.
    > >>>>>>>>> 1.b) with "static" membership, we also add the pre-defined
    > >>> member
    > >>>>> id.
    > >>>>>>>>> 1.c) with "static" membership, we also add an optional
    > >>>>>>>>> "group-change-timeout" value.
    > >>>>>>>>>
    > >>>>>>>>> 2. On the broker side, we enforce only one of the two protocols
    > >>> for
    > >>>>> all
    > >>>>>>>>> group members: we accept the protocol on the first joined
    > >> member
    > >>> of
    > >>>>> the
    > >>>>>>>>> group, and if later joining members indicate a different
    > >>> membership
    > >>>>>>>>> protocol, we reject it. If the group-change-timeout value was
    > >>>>> different
    > >>>>>>>> to
    > >>>>>>>>> the first joined member, we reject it as well.
    > >>>>>>>>
    > >>>>>>>>
    > >>>>>>>> What will happen if we have an already-deployed application that
    > >>>> wants
    > >>>>>> to
    > >>>>>>>> switch to using static membership? Let’s say there are 10
    > >>> instances
    > >>>> of
    > >>>>>> it.
    > >>>>>>>> As the instances go through a rolling restart, they will switch
    > >>> from
    > >>>>>>>> dynamic membership (the default?) to static membership. As each
    > >>> one
    > >>>>>> leaves
    > >>>>>>>> the group and restarts, they will be rejected from the group
    > >>>> (because
    > >>>>>> the
    > >>>>>>>> group is currently using dynamic membership). The group will
    > >>> shrink
    > >>>>> down
    > >>>>>>>> until there is 1 node handling all the traffic. After that one
    > >>>>> restarts,
    > >>>>>>>> the group will switch over to static membership.
    > >>>>>>>>
    > >>>>>>>> Is that right? That means that the transition plan from dynamic
    > >> to
    > >>>>>> static
    > >>>>>>>> membership isn’t very smooth.
    > >>>>>>>>
    > >>>>>>>> I’m not really sure what can be done in this case. This reminds
    > >> me
    > >>>> of
    > >>>>>> the
    > >>>>>>>> transition plans that were discussed for moving from
    > >>> zookeeper-based
    > >>>>>>>> consumers to kafka-coordinator-based consumers. That was also
    > >>> hard,
    > >>>>> and
    > >>>>>>>> ultimately we decided not to build that.
    > >>>>>>>>
    > >>>>>>>> -James
    > >>>>>>>>
    > >>>>>>>>
    > >>>>>>>
    > >>>>>>>
    > >>>>>>
    > >>>>>>
    > >>>>>
    > >>>>
    > >>>>
    > >>>>
    > >>>> --
    > >>>> -- Guozhang
    > >>>>
    > >>>
    > >>
    > >>
    > >>
    > >> --
    > >> -- Guozhang
    > >>
    >
    


Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Konstantine for correcting the details within proposal for me! Addressed them below:


> In a few places the new/proposed changes are referred to as "current".
> Which is a bit confusing considering that there is a protocol in place
> already, and by "current" someone might understand the existing one.

Fixed the `current` usage with `proposed`!

> There's the following sentence in the "Public Interfaces" section:
>"Since for many stateful consumer/stream applications, the state shuffling
> is more painful than short time partial unavailability."
> However, my understanding is that the changes proposed with KIP-345 will
> not exploit any partial availability.
We are proposing to extend session timeout inside static membership and change rebalance timeout not to remove unjoined members, which means we would detect a consumer failure slower from broker side perspective. This is what I mean by "partial unavailability": some topic partition is not making progress due to consumer dead/hanging.

> In the rejected alternatives, under point 2) I read "we can copy the member
> id to the config files". I believe it means to say "member name" unless I'm
> missing something about reusing member ids....
Updated that section!

Let me know if this makes sense to you!

Boyang
________________________________
From: Konstantine Karantasis <ko...@confluent.io>
Sent: Wednesday, November 21, 2018 2:18 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang.

Thanks for preparing this KIP! It is making good progress and will be a
great improvement for stateful Kafka applications.

Apologies for my late reply, I was away for a while. Lots of great comments
so far, so I'll probably second most of them in what I suggest below at
this point.

When I first read the KIP, I wanted to start at the end with something that
wasn't highlighted a lot. That was the topic related to handling duplicate
members. I see now that the initial suggestion of handling this situation
during offset commit has been removed, and I agree with that. Issues
related to membership seem to be handled better when the member joins the
group rather than when it tries to commit offsets. This also simplifies how
many request types need to change in order to incorporate the new member
name field.

I also agree with what Jason and Guozhang have said regarding timeouts.
Although semantically, it's easier to think of every operation having its
own timeout, operationally this can become a burden. Thus, consolidation
seems preferable here. The definition of embedded protocols on top of the
base group membership protocol for rebalancing gives enough flexibility to
address such needs in each client component separately.

Finally, some minor comments:
In a few places the new/proposed changes are referred to as "current".
Which is a bit confusing considering that there is a protocol in place
already, and by "current" someone might understand the existing one. I'd
recommend using new/proposed or equivalent when referring to changes
introduced with KIP-345 and current/existing or equivalent when referring
to existing behavior.

There's the following sentence in the "Public Interfaces" section:
"Since for many stateful consumer/stream applications, the state shuffling
is more painful than short time partial unavailability."
However, my understanding is that the changes proposed with KIP-345 will
not exploit any partial availability. A suggestion for dealing with
temporary imbalances has been made in "Incremental Cooperative Rebalancing"
which can work well with KIP-345, but here I don't see proposed changes
that suggest that some resources (e.g. partitions) will keep being used
while others will not be utilized. Thus, you might want to adjust this
sentence. Correct me if I'm missing something related to that.

In the rejected alternatives, under point 2) I read "we can copy the member
id to the config files". I believe it means to say "member name" unless I'm
missing something about reusing member ids. Also below I read: "By allowing
consumers to optionally specifying a member id" which probably implies
"member name" again. In a sense this section highlights a potential
confusion between member name and member id. I wonder if we could come up
with a better term for the new field. StaticTag, StaticLabel, or even
StaticName are some suggestions that could potentially help with confusion
between MemberId and MemberName and what corresponds to what. But I
wouldn't like to disrupt the discussion with naming conventions too much at
this point. I just mention it here as a thought.

Looking forward to see the final details of this KIP. Great work so far!

Konstantine


On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Guozhang for the great summary here, and I have been following up
> the action items here.
>
>
>   1.  I already updated the KIP to remove the expansion timeout and
> registration timeout. Great to see them being addressed in client side!
>   2.  I double checked the design and I believe that it is ok to have both
> static member and dynamic member co-exist in the same group. So the upgrade
> shouldn't be destructive and we are removing the two membership protocol
> switching APIs.
>   3.  I only have question about this one. I'm still reading the KafkaApis
> code here. Should I just use the same authorization logic for
> ForceStaticRebalanceRequest as JoinGroupRequest?
>   4.  I'm very excited to see this work with K8! Like you suggested, this
> feature could be better addressed in a separate KIP because it is pretty
> independent. I could start drafting the KIP once the current proposal is
> approved.
>   5.  I believe that we don't need fencing in offset commit request, since
> duplicate member.name issue could be handled by join group request. We
> shall reject join group with known member name but no member id (which
> means we already have an active member using this identity).
>   6.  I agree to remove that internal config once we move forward with
> static membership. And I already removed the entire section from the KIP.
>
> Let me know if you have other concerns.
>
> Best,
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Tuesday, November 20, 2018 4:21 PM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hello Boyang,
>
> Thanks a lot for the KIP! It is a great write-up and I appreciate your
> patience answering to the feedbacks from the community. I'd like to add my
> 2cents here:
>
> 1. By introducing another two timeout configs, registration_timeout and
> expansion_timeout, we are effectively having four timeout configs: session
> timeout, rebalance timeout (configured as "max.poll.interval.ms" on client
> side), and these two. Interplaying these timeout configs can be quite hard
> for users with such complexity, and hence I'm wondering if we can simplify
> the situation with as less possible timeout configs as possible. Here is a
> concrete suggestion I'd like propose:
>
> 1.a) Instead of introducing a registration_timeout in addition to the
> session_timeout for static members, we can just reuse the session_timeout
> and ask users to set it to a larger value when they are upgrading a dynamic
> client to a static client by setting the "member.name" at the same time.
> By
> default, the broker-side min.session.timeout is 6 seconds and
> max.session.timeout is 5 minutes, which seems reasonable to me (we can of
> course modify this broker config to enlarge the valid interval if we want
> in practice). And then we should also consider removing the condition for
> marking a client as failed if the rebalance timeout has reached while the
> JoinGroup was not received, so that the semantics of session_timeout and
> rebalance_timeout are totally separated: the former is only used to
> determine if a consumer member of the group should be marked as failed and
> kicked out of the group, and the latter is only used to determine the
> longest time coordinator should wait for PREPARE_REBALANCE phase. In other
> words if a member did not send the JoinGroup in time of the
> rebalance_timeout, we still include it in the new generation of the group
> and use its old subscription info to send to leader for assignment. Later
> if the member came back with HeartBeat request, we can still follow the
> normal path to bring it to the latest generation while checking that its
> sent JoinGroup request contains the same subscription info as we used to
> assign the partitions previously (which should be likely the case in
> practice). In addition, we should let static members to not send the
> LeaveGroup request when it is gracefully shutdown, so that a static member
> can only be leaving the group if its session has timed out, OR it has been
> indicated to not exist in the group any more (details below).
>
> 1.b) We have a parallel discussion about Incremental Cooperative
> Rebalancing, in which we will encode the "when to rebalance" logic at the
> application level, instead of at the protocol level. By doing this we can
> also enable a few other optimizations, e.g. at the Streams level to first
> build up the state store as standby tasks and then trigger a second
> rebalance to actually migrate the active tasks while keeping the actual
> rebalance latency and hence unavailability window to be small (
>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cbc48a38deb8649da89c908d64f14a464%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783347401510109&amp;sdata=bpN0Pb36dfXdlQn3OqAYsiXEnIJDyIwm%2BO%2FqrQ3%2BAJw%3D&amp;reserved=0).
> I'd propose we align
> KIP-345 along with this idea, and hence do not add the expansion_timeout as
> part of the protocol layer, but only do that at the application's
> coordinator / assignor layer (Connect, Streams, etc). We can still,
> deprecate the "*group.initial.rebalance.delay.ms
> <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cbc48a38deb8649da89c908d64f14a464%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783347401510109&amp;sdata=0IKKSRgDlK%2FoEmMiyv502DewhN4vButXpVNwwHgMyBI%3D&amp;reserved=0>*"
> though as part of this KIP
> since we have discussed about its limit and think it is actually not a very
> good design and could be replaced with client-side logic above.
>
>
> 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> specifically, let's say after we have upgraded broker version to be able to
> recognize the new versions of JoinGroup request and the admin requests, how
> should we upgrade the clients and enable static groups? On top of my head
> if we do a rolling bounce in which we set the member.name config as well
> as
> optionally increase the session.timeout config when we bounce each
> instance, then during this rolling bounces we will have a group contained
> with both dynamic members and static members. It means that we should have
> the group to allow such scenario (i.e. we cannot reject JoinGroup requests
> from dynamic members), and hence the "member.name" -> "member.id" mapping
> will only be partial at this scenario. Also could you describe if the
> upgrade to the first version that support this feature would ever get any
> benefits, or only the future upgrade path for rolling bounces could get
> benefits out of this feature?
>
> If that's the case and we will do 1) as suggested above, do we still need
> the enableStaticMembership and enableDynamicMembership admin requests any
> more? Seems it is not necessary any more as we will only have the notion of
> "dynamic or static members" that can co-exist in a group while there no
> notion of "dynamic or static groups", and hence these two requests are not
> needed anymore.
>
>
> 3. We need to briefly talk about the implications for ACL as we introduce
> new admin requests that are related to a specific group.id. For example,
> we
> need to make sure that whoever created the group or joined the group can
> actually send admin requests for the group, otherwise the application
> owners need to bother the Kafka operators on a multi-tenant cluster every
> time they want to send any admin requests for their groups which would be
> an operational nightmare.
>
>
> 4. I like Jason's suggestion of adding an optional field for the list of
> member names, and I'm wondering if that can be done as part of the
> forceStaticRebalance request: i.e. by passing a list of members, we will
> enforce a rebalance immediately since it indicates that some static member
> will be officially kicked out of the group and some new static members may
> be added. So back to 1.a) above, a static member can only be kicked out of
> the group if a) its session (arguably long period of time) has timed out,
> and b) this admin request explicitly state that it is no longer part of the
> group. As for execution I'm fine with keeping it as a future work of this
> KIP if you'd like to make its scope smaller.
>
> Following are minor comments:
>
> 5. I'm not sure if we need to include "member.name" as part of the
> OffsetCommitRequest for fencing purposes, as I think the memberId plus the
> generation number should be sufficient for fencing even with static
> members.
>
> 6. As mentioned above, if we agree to do 1) we can get rid of the "
> LEAVE_GROUP_ON_CLOSE_CONFIG" config.
>
>
> Guozhang
>
>
>
>
> On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
>
> > Hey Boyang,
> >
> > Thanks for the proposal! This is very useful. I have some comments below:
> >
> > 1) The motivation currently explicitly states that the goal is to improve
> > performance for heavy state application. It seems that the motivation can
> > be stronger with the following use-case. Currently for MirrorMaker
> cluster
> > with e.g. 100 MirrorMaker processes, it will take a long time to rolling
> > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> > will trigger a rebalance which currently pause the consumption of the all
> > partitions of the MirrorMaker cluster. With the change stated in this
> > patch, as long as a MirrorMaker can restart within the specified timeout
> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> for
> > leader restart) for the entire rolling bounce, which will significantly
> > improves the availability of the MirrorMaker pipeline. In my opinion, the
> > main benefit of the KIP is to avoid unnecessary rebalance if the consumer
> > process can be restarted within soon, which helps performance even if
> > overhead of state shuffling for a given process is small.
> >
> > 2) In order to simplify the KIP reading, can you follow the writeup style
> > of other KIP (e.g. KIP-98) and list the interface change such as new
> > configs (e.g. registration timeout), new request/response, new
> AdminClient
> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> > these are specified in the Proposed Change section which makes it a bit
> > inconvenient to understand the new interface that will be exposed to
> user.
> > Explanation of the current two-phase rebalance protocol probably can be
> > moved out of public interface section.
> >
> > 3) There are currently two version of JoinGroupRequest in the KIP and
> only
> > one of them has field memberId. This seems confusing.
> >
> > 4) It is mentioned in the KIP that "An admin API to force rebalance could
> > be helpful here, but we will make a call once we finished the major
> > implementation". So this seems to be still an open question in the
> current
> > design. We probably want to agree on this before voting for the KIP.
> >
> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > specify the name of the config key and the default config value? Possible
> > default values include empty string or null (similar to transaction.id
> in
> > producer config).
> >
> > 6) Regarding the use of the topic "static_member_map" to persist member
> > name map, currently if consumer coordinator broker goes offline,
> rebalance
> > is triggered and consumers will try connect to the new coordinator. If
> > these consumers can connect to the new coordinator within
> > max.poll.interval.ms which by default is 5 minutes, given that broker
> can
> > use a deterministic algorithm to determine the partition -> member_name
> > mapping, each consumer should get assigned the same set of partitions
> > without requiring state shuffling. So it is not clear whether we have a
> > strong use-case for this new logic. Can you help clarify what is the
> > benefit of using topic "static_member_map" to persist member name map?
> >
> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > mentioned that "we are using expansion timeout to replace rebalance
> > timeout, which is configured by max.poll.intervals from client side, and
> > using registration timeout to replace session timeout". Currently the
> > default max.poll.interval.ms is configured to be 5 minutes and there
> will
> > be only one rebalance if all new consumers can join within 5 minutes. So
> it
> > is not clear whether we have a strong use-case for this new config. Can
> you
> > explain what is the benefit of introducing this new config?
> >
> > 8) It is mentioned that "To distinguish between previous version of
> > protocol, we will also increase the join group request version to v4 when
> > MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> > the join group request shall be downgraded to v3 without setting the
> member
> > Id". It is probably simpler to just say that this feature is enabled if
> > JoinGroupRequest V4 is supported on both client and broker and
> MEMBER_NAME
> > is configured with non-empty string.
> >
> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> > in OffsetCommitResponse for "commit requests under static membership".
> Can
> > you clarify how broker determines whether the commit request is under
> > static membership?
> >
> > Thanks,
> > Dong
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Guozhang and Mayuresh for the followup here.
> Also I was thinking if we can have a replace API, that takes in a map of
> old to new instance Ids. Such that we can replace a consumer.
> IF we have this api, and if a consumer host goes down due to hardware
> issues, we can have another host spin up and take its place. This is like a
> cold backup which can be a step towards providing the hot backup that we
> discussed earlier in the KIP.
I like Mayuresh's suggestion, and I think we could prepare follow-up work once 345 is done to add a replace API. For the
very first version I feel this is not a must-have.

For Streams, I think we do not need an extra config for the instance id,
instead, we can re-use the way we construct the embedded consumer's client
id as:

[streams client-id] + "-StreamThread-" + [thread-id] + "-consumer"

So as long as user's specify the unique streams client-id, the resulted
consumer client-id / instance-id should be unique as well already.
So Guozhang you mean stream will enable static membership automatically correct? That would make the logic simpler
and fewer code change on stream side.

As for the LeaveGroupRequest, as I understand it, your concern is that when
we are shutting down a single Streams instance that may contain multiple
threads, shutting down that instance would mean shutting down multiple
members. Personally I'd prefer to make the LeaveGroupRequest API more
general and less inclined to Streams (I think Mayuresh also suggested
this). So I'd suggest that we keep the LeaveGroupRequest API as suggested,
i.e. a list of member.instance.ids. And in Streams we can add a new API in
KafkaStreams to expose:

1) the list of embedded consumer / producer client ids,
2) the producer's txn ids if EOS is turned on, and
3) the consumer's instance ids.
I agree with the suggestion to make the leave group request change generic. So this new Stream API
will be added on the rest layer to expose the necessary ids correct?

Looking forward to your confirmation 😊

Best,
Boyang

________________________________
From: Guozhang Wang <wa...@gmail.com>
Sent: Saturday, December 1, 2018 7:00 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

For Streams, I think we do not need an extra config for the instance id,
instead, we can re-use the way we construct the embedded consumer's client
id as:

[streams client-id] + "-StreamThread-" + [thread-id] + "-consumer"

So as long as user's specify the unique streams client-id, the resulted
consumer client-id / instance-id should be unique as well already.

As for the LeaveGroupRequest, as I understand it, your concern is that when
we are shutting down a single Streams instance that may contain multiple
threads, shutting down that instance would mean shutting down multiple
members. Personally I'd prefer to make the LeaveGroupRequest API more
general and less inclined to Streams (I think Mayuresh also suggested
this). So I'd suggest that we keep the LeaveGroupRequest API as suggested,
i.e. a list of member.instance.ids. And in Streams we can add a new API in
KafkaStreams to expose:

1) the list of embedded consumer / producer client ids,
2) the producer's txn ids if EOS is turned on, and
3) the consumer's instance ids.

So that Streams operators can read those values from KafkaStreams directly
before shutting it down and use the list in the LeaveGroupRequest API. How
about that?


Guozhang


On Fri, Nov 30, 2018 at 7:45 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> I like Guozhang's suggestion to not have to wait for session timeout in
> case we know that we want to downsize the consumer group and redistribute
> the partitions among the remaining consumers.
> IIUC, with the above suggestions, the admin api
> "removeMemberFromGroup(groupId, list[instanceId])" or
> "removeMemberFromGroup(groupId, instanceId)", will automatically cause a
> rebalance, right?
> I would prefer ist[instanceid] because that's more general scenario.
>
> Also I was thinking if we can have a replace API, that takes in a map of
> old to new instance Ids. Such that we can replace a consumer.
> IF we have this api, and if a consumer host goes down due to hardware
> issues, we can have another host spin up and take its place. This is like a
> cold backup which can be a step towards providing the hot backup that we
> discussed earlier in the KIP.
> Thoughts?
>
> Thanks,
>
> Mayuresh
>
> On Thu, Nov 29, 2018 at 1:30 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > In fact I feel that it's more convenient for user to specify a list of
> > instance id prefixes. Because
> > for general consumer application we couldn't always find a proper prefix
> > to remove a list of consumers.
> > So we are either adding list[instanceid prefix], or we could add two
> > fields: instanceid prefix, and list[instanceid]
> > for clarity purpose. As you know, two options are equivalent since full
> > name is subset of prefix.
> >
> > Let me know your thoughts!
> >
> > Boyang
> > ________________________________
> > From: Boyang Chen <bc...@outlook.com>
> > Sent: Thursday, November 29, 2018 3:39 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Thanks Guozhang for the new proposal here!
> >
> > So I'd like to propose a slightly modified version of LeaveGroupRequest:
> > instead of letting the static member consumer client themselves to send
> the
> > request (which means we still need to have some hidden configs to turn it
> > off like we did today), how about just letting any other client to send
> > this request since the LeaveGroupRequest only requires group.id and
> > member.id? So back to your operational scenarios, if some static member
> > has
> > been found crashed and it is not likely to comeback, or we simply want to
> > shrink the size of the group by shutting down some static members, we can
> > use an admin client to send the LeaveGroupRequest after the instance has
> > been completely shutdown or crashed to kick them out of the group and
> also
> > triggers the rebalance.
> >
> > One issue though, is that users may not know the member id required in
> the
> > LeaveGroupRequest. To work around it we can add the `group.instance.id`
> > along with the member id as well and then allow member id null-able. The
> > coordinator logic would then be modified as 1) if member.id is
> specified,
> > ignore instance.id and always use member.id to find the member to kick
> > out,
> > 2) otherwise, try with the instance.id to find the corresponding
> member.id
> > and kick it out, 3) if none is found, reject with an error code.
> >
> > So in sum the alternative changes are:
> >
> > a) Modify LeaveGroupRequest to add group.instance.id
> > b) Modify coordinator logic to handle such request on the broker side.
> > c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> > instanceId)" which will be translated as a LeaveGroupRequest.
> > d) [Optional] we can even batch the request by allowing
> > "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> > member.id`
> > and `instance.id` field of LeaveGroupRequest to be an array instead of a
> > single entry.
> > e) We can also remove the admin ConsumerRebalanceRequest as well for
> > simplicity (why not? paranoid of having as less request protocols as
> > possible :), as it is not needed anymore with the above proposal.
> > I agree that reusing LeaveGroupRequest is actually a good idea: we only
> > need to iterate
> > over an existing request format. Also I found that we haven't discussed
> > how we want to enable
> > this feature on Streaming applications, which is different from common
> > consumer application in that
> > Stream app uses stream thread as individual consumer.
> > For example if user specifies the client id, the stream consumer client
> id
> > will be like:
> > User client id + "-StreamThread-" + thread id + "-consumer"
> >
> > So I'm thinking we should do sth similar for defining group.instance.id
> > on Stream. We shall define another
> > config called `stream.instance.id` which would be used as prefix, and
> for
> > each thread consumer the formula
> > will look like:
> > `group.instance.id` = `stream.instance.id` + "-" + thread id +
> "-consumer"
> >
> > And for the ease of use, the interface of leave group request could
> > include `group.instance.id.prefix` instead of
> > `group.instance.id` so that we could batch remove consumers relating to
> a
> > single stream instance. This is more intuitive
> > and flexible since specifying names of 16~32 * n (n = number of stream
> > instances to shut down) consumers is not an easy
> > job without client management tooling.
> >
> > How does this workaround sound?
> >
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Thursday, November 29, 2018 2:38 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > I was thinking that with the optional static members in the admin
> > ConsumerRebalanceRequest it should be sufficient to kick out the static
> > member before their session timeout (arguably long in practice) have not
> > reached. But now I see your concern is that in some situations the admin
> > operators may not even know the full list of static members, but ONLY
> know
> > which static member has failed and hence would like to kick out of the
> > group.
> >
> > So I'd like to propose a slightly modified version of LeaveGroupRequest:
> > instead of letting the static member consumer client themselves to send
> the
> > request (which means we still need to have some hidden configs to turn it
> > off like we did today), how about just letting any other client to send
> > this request since the LeaveGroupRequest only requires group.id and
> > member.id? So back to your operational scenarios, if some static member
> > has
> > been found crashed and it is not likely to comeback, or we simply want to
> > shrink the size of the group by shutting down some static members, we can
> > use an admin client to send the LeaveGroupRequest after the instance has
> > been completely shutdown or crashed to kick them out of the group and
> also
> > triggers the rebalance.
> >
> > One issue though, is that users may not know the member id required in
> the
> > LeaveGroupRequest. To work around it we can add the `group.instance.id`
> > along with the member id as well and then allow member id null-able. The
> > coordinator logic would then be modified as 1) if member.id is
> specified,
> > ignore instance.id and always use member.id to find the member to kick
> > out,
> > 2) otherwise, try with the instance.id to find the corresponding
> member.id
> > and kick it out, 3) if none is found, reject with an error code.
> >
> > So in sum the alternative changes are:
> >
> > a) Modify LeaveGroupRequest to add group.instance.id
> > b) Modify coordinator logic to handle such request on the broker side.
> > c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> > instanceId)" which will be translated as a LeaveGroupRequest.
> > d) [Optional] we can even batch the request by allowing
> > "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> > member.id`
> > and `instance.id` field of LeaveGroupRequest to be an array instead of a
> > single entry.
> > e) We can also remove the admin ConsumerRebalanceRequest as well for
> > simplicity (why not? paranoid of having as less request protocols as
> > possible :), as it is not needed anymore with the above proposal.
> >
> >
> > WDYT?
> >
> >
> > Guozhang
> >
> > On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Guozhang and Mayuresh for the follow up! Answers are listed
> below.
> > >
> > >
> > > >  5. Regarding "So in summary, *the member will only be removed due to
> > > > session timeout*. We shall remove it from both in-memory static
> member
> > > name
> > > > mapping and member list." If the rebalance is invoked manually using
> > the
> > > > the admin apis, how long should the group coordinator wait for the
> > > members
> > > > of the group to send a JoinGroupRequest for participating in the
> > > rebalance?
> > > > How is a lagging consumer handled?
> > >
> > > Great question. Let's use c1~c4 example here:
> > >
> > >   1.  Consumer c1, c2, c3, c4 in stable state
> > >   2.  c4 goes down and we detect this issue before session timeout
> > through
> > > client monitoring. Initiate a ConsumerRebalanceRequest.
> > >   3.  A rebalance will be kicking off, and after rebalance timeout we
> > > shall keep the same assignment for c1~4, if the session timeout for c4
> > > hasn't reached
> > >   4.  Group back to stable with c1~4 (although c4 is actually offline)
> > >   5.  c4 session timeout finally reached: another rebalance triggered.
> > >
> > > For step 3, if session timeout triggered within rebalance timeout, only
> > > c1~3 will be participating in the rebalance. This is what we mean by
> > saying
> > > "rebalance
> > > timeout shall not remove current members, only session timeout will
> do."
> > > As you could see this is not an ideal scenario: we trigger extra
> > rebalance
> > > at step 5. In my reply to Guozhang I'm asking whether we should still
> use
> > > LeaveGroupRequest for static members to send a signal to broker saying
> > "I'm
> > > currently offline", and when we send ConsumerRebalanceRequest to
> broker,
> > we
> > > will actually kick off c4 because it says it's offline already, saving
> > one
> > > or multiple additional rebalances later. This way the
> > > ConsumerRebalanceRequest will be more effective in making correct
> > judgement
> > > on the group status since we have more feedback from client side.
> > >
> > > > - When we say that we would use invokeConsumerRebalance(groupId) to
> > down
> > > > scale, with the example in the above question, how will the
> > > > GroupCoordinator know that c4 should be kicked out of the group since
> > we
> > > > are trying to invoke rebalance proactively without waiting for c4's
> > > session
> > > > time out to expire. Should there be a way of telling the
> > GroupCoordinator
> > > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > > Previous proposal should be suffice to answer this question 😊
> > >
> > > - Also it looks like the statement "If the `member.id` uses
> > > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> > replace
> > > > the one within current map, if `group.member.name` is known. Also
> once
> > > we
> > > > are done with KIP-394
> > > > <
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=AWHFruanfD%2BR2S4thCniTxKUBTo9fziEzEDeefxskrs%3D&amp;reserved=0
> > > > >,
> > > > all the join group requests are requiring `member.id` to physically
> > > enter
> > > > the consumer group. This way the latest joined " is incomplete. Can
> you
> > > > take a look at this?
> > > > Also when we say "all the join group requests are requiring `
> member.id
> > `
> > > to
> > > > physically enter the consumer group." because a newly started
> consumer
> > > will
> > > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > > assigns a member.id to the newly started consumer, it has to use it
> > for
> > > > any
> > > > future JoinGroupRequests. Is my understanding correct?
> > > >
> > > Thanks for catching it! And yes, we shall use one extra round-trip
> > between
> > > consumer
> > > and broker to inform the new member id allocation.
> > >
> > > Next is the replies to Guozhang's comment:
> > > 2) I once have a discussion about the LeaveGroupRequest for static
> > members,
> > > and the reason for not having it for static members is that we'd need
> to
> > > make it a configurable behavior as well (i.e. the likelihood that a
> > static
> > > member may shutdown but come back later may be even larger than the
> > > likelihood that a shutdown static member would not come back), and
> when a
> > > shutdown is complete the instance cannot tell whether or not it will
> come
> > > back by itself. And hence letting a third party (think: admin used by
> K8s
> > > plugins) issuing a request to indicate static member changes would be
> > more
> > > plausible.
> > >
> > > I think having an optional list of all the static members that are
> still
> > in
> > > the group, rather than the members to be removed since the latter
> looks a
> > > bit less flexible to me, in the request is a good idea (remember we
> > allow a
> > > group to have both static and dynamic members at the same time, so when
> > > receiving the request, we will only do the diff and add / remove the
> > static
> > > members directly only, while still let the dynamic members to try to
> > > re-join the group with the rebalance timeout).
> > > I'm also in favor of storing all the in-group static members. In fact
> we
> > > could reuse
> > > the static membership mapping to store this information. Do you think
> > > that we should let static member send leave group request to indicate
> > > their status of "leaving",
> > > and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> > > suggesting we should
> > > remove those members when kicking off rebalance since we are shutting
> > them
> > > down already.
> > >
> > > 3) personally I favor "ids" over "names" :) Since we already have some
> > > "ids" and hence it sounds more consistent, plus on the producer side we
> > > have a `transactional.id` whose semantics is a bit similar to this
> one,
> > > i.e. for unique distinguishment of a client which may comes and goes
> but
> > > need to be persist over multiple "instance life-times".
> > > Sure we have enough votes for ids 😊I will finalize the name to `
> > > group.instance.id`, does that
> > > sound good?
> > >
> > > Best,
> > > Boyang
> > > ________________________________
> > > From: Guozhang Wang <wa...@gmail.com>
> > > Sent: Wednesday, November 28, 2018 4:51 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Regarding Jason's question and Boyang's responses:
> > >
> > > 2) I once have a discussion about the LeaveGroupRequest for static
> > members,
> > > and the reason for not having it for static members is that we'd need
> to
> > > make it a configurable behavior as well (i.e. the likelihood that a
> > static
> > > member may shutdown but come back later may be even larger than the
> > > likelihood that a shutdown static member would not come back), and
> when a
> > > shutdown is complete the instance cannot tell whether or not it will
> come
> > > back by itself. And hence letting a third party (think: admin used by
> K8s
> > > plugins) issuing a request to indicate static member changes would be
> > more
> > > plausible.
> > >
> > > I think having an optional list of all the static members that are
> still
> > in
> > > the group, rather than the members to be removed since the latter
> looks a
> > > bit less flexible to me, in the request is a good idea (remember we
> > allow a
> > > group to have both static and dynamic members at the same time, so when
> > > receiving the request, we will only do the diff and add / remove the
> > static
> > > members directly only, while still let the dynamic members to try to
> > > re-join the group with the rebalance timeout).
> > >
> > > 3) personally I favor "ids" over "names" :) Since we already have some
> > > "ids" and hence it sounds more consistent, plus on the producer side we
> > > have a `transactional.id` whose semantics is a bit similar to this
> one,
> > > i.e. for unique distinguishment of a client which may comes and goes
> but
> > > need to be persist over multiple "instance life-times".
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the replies. Please find the follow up queries below.
> > > >
> > > >     5. Regarding "So in summary, *the member will only be removed due
> > to
> > > > session timeout*. We shall remove it from both in-memory static
> member
> > > name
> > > > mapping and member list." If the rebalance is invoked manually using
> > the
> > > > the admin apis, how long should the group coordinator wait for the
> > > members
> > > > of the group to send a JoinGroupRequest for participating in the
> > > rebalance?
> > > > How is a lagging consumer handled?
> > > > The plan is to disable member kick out when rebalance.timeout is
> > reached,
> > > > so basically we are not "waiting" any
> > > > join group request from existing members; we shall just rebalance
> base
> > on
> > > > what we currently have within the group
> > > > metadata. Lagging consumer will trigger rebalance later if session
> > > timeout
> > > > > rebalance timeout.
> > > >
> > > > >
> > > > Just wanted to understand this better. Lets take an example, say we
> > have
> > > a
> > > > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > > > Everything is running fine and suddenly C4 host has issues and it
> > goes
> > > > > down. Now we notice that we can still operate with c1, c2, c3 and
> > don't
> > > > > want to wait for
> > > > > c4 to come back up. We use the admin api
> > > > > "invokeConsumerRebalance("GroupA")".
> > > > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join
> the
> > > > > group again (in there heartBeatResponse) as first step of
> rebalance.
> > > > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3
> > is
> > > > > delayed. At this stage, if we are not "waiting" on any join group
> > > > request,
> > > > > few things can happen :
> > > > >
> > > > >    - c4's partitions are distributed only among c1,c2. c3 maintains
> > its
> > > > >    original assignment. c1, c2 will start processing the newly
> > assigned
> > > > >    partitions.
> > > > >
> > > > > OR
> > > > >
> > > > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2
> > start
> > > > >    processing the newly assigned partitions. c3 gets to know about
> > the
> > > > newly
> > > > >    assigned partitions later when it sends the JoinGroupRequest
> > (which
> > > > was
> > > > >    delayed).
> > > > >
> > > > > OR
> > > > >
> > > > >    - Will the rebalance do a complete reassignment, where c1, c2,
> c3
> > > have
> > > > >    to give up there partitions and all the partitions belonging to
> > c1,
> > > > c2, c3,
> > > > >    c4 will be redistributed among c1, c2, c3 ? If this is the case,
> > the
> > > > >    GroupCoordinator needs to give some buffer time for c1, c2, c3
> to
> > > > revoke
> > > > >    there partitions and rejoin the group.
> > > > >
> > > > > This is as per my understanding of how the KIP would work without
> > > > changing
> > > > > the underlying group coordination workflow. Please correct me if I
> > > > > misunderstood something here.
> > > > >
> > > >
> > > >
> > > > - When we say that we would use invokeConsumerRebalance(groupId) to
> > down
> > > > scale, with the example in the above question, how will the
> > > > GroupCoordinator know that c4 should be kicked out of the group since
> > we
> > > > are trying to invoke rebalance proactively without waiting for c4's
> > > session
> > > > time out to expire. Should there be a way of telling the
> > GroupCoordinator
> > > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > > >
> > > > - Also it looks like the statement "If the `member.id` uses
> > > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> > replace
> > > > the one within current map, if `group.member.name` is known. Also
> once
> > > we
> > > > are done with KIP-394
> > > > <
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=AWHFruanfD%2BR2S4thCniTxKUBTo9fziEzEDeefxskrs%3D&amp;reserved=0
> > > > >,
> > > > all the join group requests are requiring `member.id` to physically
> > > enter
> > > > the consumer group. This way the latest joined " is incomplete. Can
> you
> > > > take a look at this?
> > > > Also when we say "all the join group requests are requiring `
> member.id
> > `
> > > to
> > > > physically enter the consumer group." because a newly started
> consumer
> > > will
> > > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > > assigns a member.id to the newly started consumer, it has to use it
> > for
> > > > any
> > > > future JoinGroupRequests. Is my understanding correct?
> > > >
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> > > both
> > > > > in this reply.
> > > > >
> > > > >
> > > > > >    1. Do you intend to have member.id is a static config like
> > > > > member.name
> > > > > >    after KIP-345 and KIP-394?
> > > > >
> > > > > No, we shall only rely on broker to allocate member.id for the
> > > consumer
> > > > > instances. FYI, I already
> > > > >
> > > > > started the discussion thread for KIP-394 😊
> > > > >
> > > > > >    2. Regarding "On client side, we add a new config called
> > > MEMBER_NAME
> > > > > in
> > > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > > config
> > > > is
> > > > > > set,
> > > > > >    we will put it in the initial join group request to identify
> > > itself
> > > > > as a
> > > > > >    static member (static membership); otherwise, we will still
> send
> > > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > > (dynamic
> > > > > >    membership)."
> > > > > >       - What is the value of member_id sent in the first
> > > > JoinGroupRequest
> > > > > >       when member_name is set (using static rebalance)? Is it
> > > > > > UNKNOW_MEMBER_ID?
> > > > >
> > > > > Yes, we could only use unknown member id. Actually this part of the
> > > > > proposal is outdated,
> > > > >
> > > > > let me do another audit of the whole doc. Basically, it is
> currently
> > > > > impossible to send `member.id`
> > > > >
> > > > > when consumer restarted. Sorry for the confusions!
> > > > >
> > > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > > match
> > > > > the
> > > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > > edge
> > > > > case
> > > > > >    that if we could have members with the same `member.name`
> (for
> > > > > example
> > > > > >    mis-configured instances with a valid member.id but added a
> > used
> > > > > member
> > > > > >    name on runtime). When member name has duplicates, we could
> > refuse
> > > > > join
> > > > > >    request from members with an outdated `member.id` (since we
> > > update
> > > > > the
> > > > > >    mapping upon each join group request). In an edge case where
> the
> > > > > client
> > > > > >    hits this exception in the response, it is suggesting that
> some
> > > > other
> > > > > >    consumer takes its spot."
> > > > > >       - The part of "some other consumer takes the spot" would be
> > > > > >       intentional, right? Also when you say " The edge case that
> if
> > > we
> > > > > >       could have members with the same `member.name` (for
> example
> > > > > >       mis-configured instances *with a valid member.id <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1bCMsB35UGHNRiI5c9KAzd34wAwiGs7v2OVmWKfRAC0%3D&amp;reserved=0
> > > > > >
> > > > > > *but
> > > > > >       added a used member name on runtime).", what do you mean by
> > > > *valid
> > > > > >       member id* here? Does it mean that there exist a mapping of
> > > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > > member.name
> > > > > >       <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1zOe9ZqIHB7lvt8XJt2jtLvRaP75G3OamvtLRLjysyo%3D&amp;reserved=0
> > > > >
> > > > > = MemberB and member.id <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1bCMsB35UGHNRiI5c9KAzd34wAwiGs7v2OVmWKfRAC0%3D&amp;reserved=0
> > > > >
> > > > > =
> > > > > > id1 *
> > > > > >       ?
> > > > >
> > > > > I would take Jason's advice that each time we have unknown member
> > > joining
> > > > > the group, the broker will
> > > > >
> > > > > always assign a new and unique id to track its identity. In this
> way,
> > > > > consumer with duplicate member name
> > > > >
> > > > > will be fenced.
> > > > >
> > > > > >    4. Depending on your explanation for point 2 and the point 3
> > above
> > > > > >    regarding returning back MEMBER_ID_MISMATCH on having a
> matching
> > > > > >    member_name but unknown member_id, if the consumer sends
> > > > > > "UNKNOW_MEMBER_ID"
> > > > > >    on the first JoinGroupRequest and relies on the
> GroupCoordinator
> > > to
> > > > > > give it
> > > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > > >
> > > > > Like explained above, we shall not materialize the member.id.
> > Instead
> > > we
> > > > > need to rely on broker to allocate
> > > > >
> > > > > a unique id for consumer just like what we have now.
> > > > >
> > > > > >    5. Regarding "So in summary, *the member will only be removed
> > due
> > > to
> > > > > >    session timeout*. We shall remove it from both in-memory
> static
> > > > member
> > > > > >    name mapping and member list."
> > > > > >       - If the rebalance is invoked manually using the the admin
> > > apis,
> > > > > how
> > > > > >       long should the group coordinator wait for the members of
> the
> > > > > > group to send
> > > > > >       a JoinGroupRequest for participating in the rebalance? How
> > is a
> > > > > > lagging
> > > > > >       consumer handled?
> > > > >
> > > > > The plan is to disable member kick out when rebalance.timeout is
> > > reached,
> > > > > so basically we are not "waiting" any
> > > > >
> > > > > join group request from existing members; we shall just rebalance
> > base
> > > on
> > > > > what we currently have within the group
> > > > >
> > > > > metadata. Lagging consumer will trigger rebalance later if session
> > > > timeout
> > > > > > rebalance timeout.
> > > > >
> > > > > >    6. Another detail to take care is that we need to
> automatically
> > > take
> > > > > the
> > > > > >    hash of group id so that we know which broker to send this
> > request
> > > > to.
> > > > > >       - I assume this should be same as the way we find the
> > > > coordinator,
> > > > > >       today right? If yes, should we specify it in the KIP ?
> > > > >
> > > > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > > > >
> > > > > >    7. Are there any specific failure scenarios when you say
> "other
> > > > > >    potential failure cases."? It would be good to mention them
> > > > > explicitly,
> > > > > > if
> > > > > >    you think there are any.
> > > > >
> > > > > Nah, I'm gonna remove it because it seems causing more confusion
> than
> > > > > making my assumption clear, which is
> > > > >
> > > > > "there could be other failure cases that I can't enumerate now" 😊
> > > > >
> > > > > >    8. It would be good to have a rollback plan as you have for
> roll
> > > > > forward
> > > > > >    in the KIP.
> > > > >
> > > > > Great suggestion! Added a simple rollback plan.
> > > > >
> > > > >
> > > > > Next is answering Jason's suggestions:
> > > > >
> > > > > 1. This may be the same thing that Mayuresh is asking about. I
> think
> > > the
> > > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > > member
> > > > > name, but no member id, then we will return the current member id
> > > > > associated with that name. It seems in this case that we wouldn't
> be
> > > able
> > > > > to protect from having two consumers active with the same
> configured
> > > > > member.name? For example, imagine that we had a consumer with
> > > > member.name
> > > > > =A
> > > > > which is assigned member.id=1. Suppose it becomes a zombie and a
> new
> > > > > instance starts up with member.name=A. If it is also assigned
> > > member.id
> > > > =1,
> > > > > then how can we detect the zombie if it comes back to life? Both
> > > > instances
> > > > > will have the same member.id.
> > > > >
> > > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > > need
> > > > to
> > > > > fence previous members. I am wondering if we can generate a new
> > > > member.id
> > > > > every time we receive a request from a static member with an
> unknown
> > > > member
> > > > > id. If the old instance with the same member.name attempts any
> > > > operation,
> > > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> > the
> > > > > subscription of the new instance hasn't changed, then we can skip
> the
> > > > > rebalance and return the current assignment without forcing a
> > > rebalance.
> > > > >
> > > > > The trick to making this work is in the error handling of the
> zombie
> > > > > consumer. If the zombie simply resets its member.id and rejoins to
> > > get a
> > > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would
> end
> > > up
> > > > > fencing the new member. We want to avoid this. There needs to be an
> > > > > expectation for static members that the member.id of a static
> member
> > > > will
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the
> > > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > > consumers
> > > > > with static member names.
> > > > >
> > > > > Yep, I like this idea! Keep giving out refresh member.id when
> facing
> > > > > anonymous request will definitely
> > > > >
> > > > > prevent processing bug due to duplicate consumers, however I don't
> > > think
> > > > I
> > > > > fully understand the 3rd paragraph where
> > > > >
> > > > > you mentioned  "There needs to be an expectation for static members
> > > that
> > > > > the member.id of a static member will
> > > > >
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the group. "  How do you plan
> > > > > to know whether this member is new member or old member? I feel
> even
> > > with
> > > > > zombie consumer takes the ownership,
> > > > > it should be detected very quickly (as MISMATCH_ID exception
> trigger
> > > > > original consumer instance dies)
> > > > > and end user will start to fix it right away. Is there any similar
> > > logic
> > > > > we applied in fencing duplicate `transaction.id`?
> > > > >
> > > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me.
> As
> > > far
> > > > as
> > > > > I understand it, it is used for scaling down a consumer group and
> > > somehow
> > > > > bypasses normal session timeout expiration. I am wondering how
> > critical
> > > > > this piece is and whether we can leave it for future work. If not,
> > then
> > > > it
> > > > > would be helpful to elaborate on its implementation. How would the
> > > > > coordinator know which members to kick out of the group?
> > > > >
> > > > > This API is needed when we need to immediately trigger rebalance
> > > instead
> > > > > of waiting session timeout
> > > > >
> > > > > or rebalance timeout (Emergent scale up/down). It is very necessary
> > to
> > > > > have it for
> > > > >
> > > > > management purpose because user could choose when to trigger
> > rebalance
> > > > > pretty freely,
> > > > >
> > > > > gaining more client side control.
> > > > >
> > > > > In the meanwhile I see your point that we need to actually have the
> > > > > ability to kick out members that we plan
> > > > >
> > > > > to scale down fast (as rebalance timeout no longer kicks any
> offline
> > > > > member out of the group), I will think of adding an optional
> > > > >
> > > > > list of members that are ready to be removed.
> > > > >
> > > > > Another idea is to let static member send `LeaveGroupRequest` when
> > they
> > > > > are going offline (either scale down or bouncing),
> > > > >
> > > > > and broker will cache this information as "OfflineMembers" without
> > > > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> > > will
> > > > >
> > > > > kick the static members that are currently offline and trigger
> > > rebalance
> > > > > immediately. How does this plan sound?
> > > > >
> > > > > 3. I've been holding back on mentioning this, but I think we should
> > > > > reconsider the name `member.name`. I think we want something that
> > > > suggests
> > > > > its expectation of uniqueness in the group. How about `
> > > group.instance.id
> > > > `
> > > > > to go along with `group.id`?
> > > > >
> > > > > Yea, Dong and Stanislav also mentioned this naming. I personally
> buy
> > in
> > > > > the namespace idea, and
> > > > >
> > > > > since we already use `member.name` in a lot of context, I decide
> to
> > > > > rename the config to `group.member.name`
> > > > >
> > > > > which should be sufficient for solving all the concerns we have
> now.
> > > > > Sounds good?
> > > > >
> > > > >
> > > > > Thank you for your great suggestions! Let me know if my reply makes
> > > sense
> > > > > her.
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Jason Gustafson <ja...@confluent.io>
> > > > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > > > To: dev
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the updates. Looks like we're headed in the right
> > direction
> > > > and
> > > > > clearly the interest that this KIP is receiving shows how strong
> the
> > > > > motivation is!
> > > > >
> > > > > I have a few questions:
> > > > >
> > > > > 1. This may be the same thing that Mayuresh is asking about. I
> think
> > > the
> > > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > > member
> > > > > name, but no member id, then we will return the current member id
> > > > > associated with that name. It seems in this case that we wouldn't
> be
> > > able
> > > > > to protect from having two consumers active with the same
> configured
> > > > > member.name? For example, imagine that we had a consumer with
> > > > member.name
> > > > > =A
> > > > > which is assigned member.id=1. Suppose it becomes a zombie and a
> new
> > > > > instance starts up with member.name=A. If it is also assigned
> > > member.id
> > > > =1,
> > > > > then how can we detect the zombie if it comes back to life? Both
> > > > instances
> > > > > will have the same member.id.
> > > > >
> > > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > > need
> > > > to
> > > > > fence previous members. I am wondering if we can generate a new
> > > > member.id
> > > > > every time we receive a request from a static member with an
> unknown
> > > > member
> > > > > id. If the old instance with the same member.name attempts any
> > > > operation,
> > > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> > the
> > > > > subscription of the new instance hasn't changed, then we can skip
> the
> > > > > rebalance and return the current assignment without forcing a
> > > rebalance.
> > > > >
> > > > > The trick to making this work is in the error handling of the
> zombie
> > > > > consumer. If the zombie simply resets its member.id and rejoins to
> > > get a
> > > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would
> end
> > > up
> > > > > fencing the new member. We want to avoid this. There needs to be an
> > > > > expectation for static members that the member.id of a static
> member
> > > > will
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the
> > > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > > consumers
> > > > > with static member names.
> > > > >
> > > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me.
> As
> > > far
> > > > as
> > > > > I understand it, it is used for scaling down a consumer group and
> > > somehow
> > > > > bypasses normal session timeout expiration. I am wondering how
> > critical
> > > > > this piece is and whether we can leave it for future work. If not,
> > then
> > > > it
> > > > > would be helpful to elaborate on its implementation. How would the
> > > > > coordinator know which members to kick out of the group?
> > > > >
> > > > > 3. I've been holding back on mentioning this, but I think we should
> > > > > reconsider the name `member.name`. I think we want something that
> > > > suggests
> > > > > its expectation of uniqueness in the group. How about `
> > > group.instance.id
> > > > `
> > > > > to go along with `group.id`?
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > > > gharatmayuresh15@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > Thanks a lot for replying to all the queries and discussions
> here,
> > so
> > > > > > patiently.
> > > > > > Really appreciate it.
> > > > > >
> > > > > > Had a few questions and suggestions after rereading the current
> > > version
> > > > > of
> > > > > > the KIP :
> > > > > >
> > > > > >
> > > > > >    1. Do you intend to have member.id is a static config like
> > > > > member.name
> > > > > >    after KIP-345 and KIP-394?
> > > > > >    2. Regarding "On client side, we add a new config called
> > > MEMBER_NAME
> > > > > in
> > > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > > config
> > > > is
> > > > > > set,
> > > > > >    we will put it in the initial join group request to identify
> > > itself
> > > > > as a
> > > > > >    static member (static membership); otherwise, we will still
> send
> > > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > > (dynamic
> > > > > >    membership)."
> > > > > >       - What is the value of member_id sent in the first
> > > > JoinGroupRequest
> > > > > >       when member_name is set (using static rebalance)? Is it
> > > > > > UNKNOW_MEMBER_ID?
> > > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > > match
> > > > > the
> > > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > > edge
> > > > > case
> > > > > >    that if we could have members with the same `member.name`
> (for
> > > > > example
> > > > > >    mis-configured instances with a valid member.id but added a
> > used
> > > > > member
> > > > > >    name on runtime). When member name has duplicates, we could
> > refuse
> > > > > join
> > > > > >    request from members with an outdated `member.id` (since we
> > > update
> > > > > the
> > > > > >    mapping upon each join group request). In an edge case where
> the
> > > > > client
> > > > > >    hits this exception in the response, it is suggesting that
> some
> > > > other
> > > > > >    consumer takes its spot."
> > > > > >       - The part of "some other consumer takes the spot" would be
> > > > > >       intentional, right? Also when you say " The edge case that
> if
> > > we
> > > > > >       could have members with the same `member.name` (for
> example
> > > > > >       mis-configured instances *with a valid member.id <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1bCMsB35UGHNRiI5c9KAzd34wAwiGs7v2OVmWKfRAC0%3D&amp;reserved=0
> > > > > >
> > > > > > *but
> > > > > >       added a used member name on runtime).", what do you mean by
> > > > *valid
> > > > > >       member id* here? Does it mean that there exist a mapping of
> > > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > > member.name
> > > > > >       <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1zOe9ZqIHB7lvt8XJt2jtLvRaP75G3OamvtLRLjysyo%3D&amp;reserved=0
> > > > >
> > > > > = MemberB and member.id <
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1bCMsB35UGHNRiI5c9KAzd34wAwiGs7v2OVmWKfRAC0%3D&amp;reserved=0
> > > > >
> > > > > =
> > > > > > id1 *
> > > > > >       ?
> > > > > >    4. Depending on your explanation for point 2 and the point 3
> > above
> > > > > >    regarding returning back MEMBER_ID_MISMATCH on having a
> matching
> > > > > >    member_name but unknown member_id, if the consumer sends
> > > > > > "UNKNOW_MEMBER_ID"
> > > > > >    on the first JoinGroupRequest and relies on the
> GroupCoordinator
> > > to
> > > > > > give it
> > > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > > > >    5. Regarding "So in summary, *the member will only be removed
> > due
> > > to
> > > > > >    session timeout*. We shall remove it from both in-memory
> static
> > > > member
> > > > > >    name mapping and member list."
> > > > > >       - If the rebalance is invoked manually using the the admin
> > > apis,
> > > > > how
> > > > > >       long should the group coordinator wait for the members of
> the
> > > > > > group to send
> > > > > >       a JoinGroupRequest for participating in the rebalance? How
> > is a
> > > > > > lagging
> > > > > >       consumer handled?
> > > > > >    6. Another detail to take care is that we need to
> automatically
> > > take
> > > > > the
> > > > > >    hash of group id so that we know which broker to send this
> > request
> > > > to.
> > > > > >       - I assume this should be same as the way we find the
> > > > coordinator,
> > > > > >       today right? If yes, should we specify it in the KIP ?
> > > > > >    7. Are there any specific failure scenarios when you say
> "other
> > > > > >    potential failure cases."? It would be good to mention them
> > > > > explicitly,
> > > > > > if
> > > > > >    you think there are any.
> > > > > >    8. It would be good to have a rollback plan as you have for
> roll
> > > > > forward
> > > > > >    in the KIP.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Mayuresh
> > > > > >
> > > > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > > > gharatmayuresh15@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Boyang,
> > > > > > >
> > > > > > > Do you have a discuss thread for KIP-394 that you mentioned
> here
> > ?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Mayuresh
> > > > > > >
> > > > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <
> bchen11@outlook.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > >> Hey Dong, thanks for the follow-up here!
> > > > > > >>
> > > > > > >>
> > > > > > >> 1) It is not very clear to the user what is the difference
> > between
> > > > > > >> member.name and client.id as both seems to be used to
> identify
> > > the
> > > > > > >> consumer. I am wondering if it would be more intuitive to name
> > it
> > > > > > >> group.member.name (preferred choice since it matches the
> > current
> > > > > > group.id
> > > > > > >> config name) or rebalance.member.name to explicitly show that
> > the
> > > > id
> > > > > is
> > > > > > >> solely used for rebalance.
> > > > > > >> Great question. I feel `member.name` is enough to explain
> > itself,
> > > > it
> > > > > > >> seems not very
> > > > > > >> helpful to make the config name longer. Comparing `name` with
> > `id`
> > > > > gives
> > > > > > >> user the
> > > > > > >> impression that they have the control over it with customized
> > rule
> > > > > than
> > > > > > >> library decided.
> > > > > > >>
> > > > > > >> 2) In the interface change section it is said that
> > > > > > >> GroupMaxSessionTimeoutMs
> > > > > > >> will be changed to 30 minutes. It seems to suggest that we
> will
> > > > change
> > > > > > the
> > > > > > >> default value of this config. It does not seem necessary to
> > > increase
> > > > > the
> > > > > > >> time of consumer failure detection when user doesn't use
> static
> > > > > > >> membership.
> > > > > > >> Also, say static membership is enabled, then this default
> config
> > > > > change
> > > > > > >> will cause a partition to be unavailable for consumption for
> 30
> > > > > minutes
> > > > > > if
> > > > > > >> there is hard consumer failure, which seems to be worse
> > experience
> > > > > than
> > > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > > particularly
> > > > > > >> for
> > > > > > >> new users of Kafka. Could you explain more why we should make
> > this
> > > > > > change?
> > > > > > >> We are not changing the default session timeout value. We are
> > just
> > > > > > >> changing the
> > > > > > >> cap we are enforcing on the session timeout max value. So this
> > > > change
> > > > > is
> > > > > > >> not affecting
> > > > > > >> what kind of membership end user is using, and loosing the cap
> > is
> > > > > giving
> > > > > > >> end user
> > > > > > >> more flexibility on trade-off between liveness and stability.
> > > > > > >>
> > > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > > DUPLICATE_STATIC_MEMBER
> > > > > > >> into one error? It seems that these two errors are currently
> > > handled
> > > > > by
> > > > > > >> the
> > > > > > >> consumer in the same way. And we don't also don't expect
> > > > > > >> MEMBER_ID_MISMATCH
> > > > > > >> to happen. Thus it is not clear what is the benefit of having
> > two
> > > > > > errors.
> > > > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> > > because
> > > > > with
> > > > > > >> the KIP-394<
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=OcUHb3tNzXVGxruhgofPDya5GpoJqFDfyq0jiIffZMY%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> we will automatically fence all join requests with
> > > > UNKNOWN_MEMBER_ID.
> > > > > > >>
> > > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join
> group
> > > > > > contains
> > > > > > >> member name which is already in the consumer group, however
> the
> > > > member
> > > > > > id
> > > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > > >> JoinGroupRequest with an existing memberName (as the
> coordinator
> > > has
> > > > > not
> > > > > > >> expired this member from the memory) and memberId
> > > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > > persisted
> > > > > > >> across consumer restart in the consumer side). Does it mean
> that
> > > > > > >> JoinGroupRequest from a newly restarted consumer will always
> be
> > > > > rejected
> > > > > > >> until the sessionTimeoutMs has passed?
> > > > > > >> Same answer as question 3). This part of the logic shall be
> > > removed
> > > > > from
> > > > > > >> the proposal.
> > > > > > >>
> > > > > > >> 5) It seems that we always add two methods to the interface
> > > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> > options
> > > > and
> > > > > > the
> > > > > > >> other without option. Could this be specified in the interface
> > > > change
> > > > > > >> section?
> > > > > > >> Sounds good! Added both methods.
> > > > > > >>
> > > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE
> to
> > > > > trigger
> > > > > > >> rebalance? If so, we probably want to specify the command line
> > > tool
> > > > > > >> interface similar to
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=egpgN2DcSoFFLeOrGRv9EgtAuMLUxrvazXsUOIKWsGE%3D&amp;reserved=0
> > > > > > >> .
> > > > > > >> Added the script.
> > > > > > >>
> > > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> > with
> > > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> > extra
> > > > > > meaning
> > > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > > seems
> > > > > > >> simpler
> > > > > > >> to allows this API to trigger rebalance regardless of whether
> > > > consumer
> > > > > > is
> > > > > > >> configured with memberName.
> > > > > > >> Sounds good. Right now I feel for both static and dynamic
> > > membership
> > > > > it
> > > > > > is
> > > > > > >> more manageable to introduce the consumer rebalance method
> > through
> > > > > admin
> > > > > > >> client API.
> > > > > > >>
> > > > > > >> 8) It is not very clear how the newly added AdminClient API
> > > trigger
> > > > > > >> rebalance. For example, does it send request? Can this be
> > > explained
> > > > in
> > > > > > the
> > > > > > >> KIP?
> > > > > > >>
> > > > > > >> Sure, I will add more details to the API.
> > > > > > >>
> > > > > > >>
> > > > > > >> Thanks again for the helpful suggestions!
> > > > > > >>
> > > > > > >>
> > > > > > >> Best,
> > > > > > >> Boyang
> > > > > > >>
> > > > > > >> ________________________________
> > > > > > >> From: Dong Lin <li...@gmail.com>
> > > > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > > > >> To: dev
> > > > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > by
> > > > > > >> specifying member id
> > > > > > >>
> > > > > > >> Hey Boyang,
> > > > > > >>
> > > > > > >> Thanks for the update! Here are some followup comments:
> > > > > > >>
> > > > > > >> 1) It is not very clear to the user what is the difference
> > between
> > > > > > >> member.name and client.id as both seems to be used to
> identify
> > > the
> > > > > > >> consumer. I am wondering if it would be more intuitive to name
> > it
> > > > > > >> group.member.name (preferred choice since it matches the
> > current
> > > > > > group.id
> > > > > > >> config name) or rebalance.member.name to explicitly show that
> > the
> > > > id
> > > > > is
> > > > > > >> solely used for rebalance.
> > > > > > >>
> > > > > > >> 2) In the interface change section it is said that
> > > > > > >> GroupMaxSessionTimeoutMs
> > > > > > >> will be changed to 30 minutes. It seems to suggest that we
> will
> > > > change
> > > > > > the
> > > > > > >> default value of this config. It does not seem necessary to
> > > increase
> > > > > the
> > > > > > >> time of consumer failure detection when user doesn't use
> static
> > > > > > >> membership.
> > > > > > >> Also, say static membership is enabled, then this default
> config
> > > > > change
> > > > > > >> will cause a partition to be unavailable for consumption for
> 30
> > > > > minutes
> > > > > > if
> > > > > > >> there is hard consumer failure, which seems to be worse
> > experience
> > > > > than
> > > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > > particularly
> > > > > > >> for
> > > > > > >> new users of Kafka. Could you explain more why we should make
> > this
> > > > > > change?
> > > > > > >>
> > > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > > DUPLICATE_STATIC_MEMBER
> > > > > > >> into one error? It seems that these two errors are currently
> > > handled
> > > > > by
> > > > > > >> the
> > > > > > >> consumer in the same way. And we don't also don't expect
> > > > > > >> MEMBER_ID_MISMATCH
> > > > > > >> to happen. Thus it is not clear what is the benefit of having
> > two
> > > > > > errors.
> > > > > > >>
> > > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join
> group
> > > > > > contains
> > > > > > >> member name which is already in the consumer group, however
> the
> > > > member
> > > > > > id
> > > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > > >> JoinGroupRequest with an existing memberName (as the
> coordinator
> > > has
> > > > > not
> > > > > > >> expired this member from the memory) and memberId
> > > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > > persisted
> > > > > > >> across consumer restart in the consumer side). Does it mean
> that
> > > > > > >> JoinGroupRequest from a newly restarted consumer will always
> be
> > > > > rejected
> > > > > > >> until the sessionTimeoutMs has passed?
> > > > > > >>
> > > > > > >> 5) It seems that we always add two methods to the interface
> > > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> > options
> > > > and
> > > > > > the
> > > > > > >> other without option. Could this be specified in the interface
> > > > change
> > > > > > >> section?
> > > > > > >>
> > > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE
> to
> > > > > trigger
> > > > > > >> rebalance? If so, we probably want to specify the command line
> > > tool
> > > > > > >> interface similar to
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=egpgN2DcSoFFLeOrGRv9EgtAuMLUxrvazXsUOIKWsGE%3D&amp;reserved=0
> > > > > > >> .
> > > > > > >>
> > > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> > with
> > > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> > extra
> > > > > > meaning
> > > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > > seems
> > > > > > >> simpler
> > > > > > >> to allows this API to trigger rebalance regardless of whether
> > > > consumer
> > > > > > is
> > > > > > >> configured with memberName.
> > > > > > >>
> > > > > > >> 8) It is not very clear how the newly added AdminClient API
> > > trigger
> > > > > > >> rebalance. For example, does it send request? Can this be
> > > explained
> > > > in
> > > > > > the
> > > > > > >> KIP?
> > > > > > >>
> > > > > > >> Thanks,
> > > > > > >> Dong
> > > > > > >>
> > > > > > >>
> > > > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <
> > bchen11@outlook.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Hey Mayuresh,
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > thanks for your feedbacks! I will try do another checklist
> > here.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > By this you mean, even if the application has not called
> > > > > > >> > > KafkaConsumer.poll() within session timeout, it will not
> be
> > > > > sending
> > > > > > >> the
> > > > > > >> > > LeaveGroup request, right?
> > > > > > >> >
> > > > > > >> > Yep it's true, we will prevent client from sending leave
> group
> > > > > request
> > > > > > >> > when they are set with `member.name`.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > When is the member.name removed from this map?
> > > > > > >> > Good question, we will only kick off member due to session
> > > timeout
> > > > > > >> within
> > > > > > >> > static membership. Let me update the KIP to clearly assert
> > that.
> > > > > > >> >
> > > > > > >> > > How is this case (missing member id) handled on the client
> > > side?
> > > > > > What
> > > > > > >> is
> > > > > > >> > the application that
> > > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > > >> > I have extended the two exceptions within join group
> response
> > > V4.
> > > > > > >> > Basically I define both corresponding actions to be
> immediate
> > > > > failing
> > > > > > >> > client application, because so far it is unknown what kind
> of
> > > > client
> > > > > > >> issue
> > > > > > >> > could trigger them. After the first version, we will keep
> > > enhance
> > > > > the
> > > > > > >> error
> > > > > > >> > handling logic!
> > > > > > >> >
> > > > > > >> > > This would mean that it might take more time to detect
> > unowned
> > > > > topic
> > > > > > >> > > partitions and may cause delay for applications that
> perform
> > > > data
> > > > > > >> > mirroring
> > > > > > >> > > tasks. I discussed this with our sre and we have a
> > suggestion
> > > to
> > > > > > make
> > > > > > >> > here
> > > > > > >> > > as listed below separately.
> > > > > > >> > The goal of extending session timeout cap is for users with
> > good
> > > > > > client
> > > > > > >> > side monitoring tools that could auto-heal the dead
> consumers
> > > very
> > > > > > >> fast. So
> > > > > > >> > it is optional (and personal) to extend session timeout to a
> > > > > > reasonable
> > > > > > >> > number with different client scenarios.
> > > > > > >> >
> > > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > > > >> >
> > > > > > >> > > What do you mean by " Internally we would optimize this
> > logic
> > > by
> > > > > > >> having
> > > > > > >> > > rebalance timeout only in charge of stopping prepare
> > rebalance
> > > > > > stage,
> > > > > > >> > > without removing non-responsive members immediately."
> There
> > > > would
> > > > > > not
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > > request
> > > > > > later,
> > > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > > > functionality
> > > > > > >> > to only use as a timer to
> > > > > > >> > end prepare rebalance stage. This way, late joining static
> > > members
> > > > > > will
> > > > > > >> > not trigger further rebalance
> > > > > > >> > as long as they are within session timeout. I added your
> > > highlight
> > > > > to
> > > > > > >> the
> > > > > > >> > KIP!
> > > > > > >> >
> > > > > > >> > > The KIP talks about scale up scenario but its not quite
> > clear
> > > > how
> > > > > we
> > > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> > we
> > > > > adding
> > > > > > >> > status
> > > > > > >> > > "learner" ?. Can you shed more light on how this is
> handled
> > in
> > > > the
> > > > > > >> KIP,
> > > > > > >> > if
> > > > > > >> > > its handled?
> > > > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> > > because
> > > > we
> > > > > > >> > believe client side could
> > > > > > >> > better handle this logic.
> > > > > > >> >
> > > > > > >> > > I think Jason had brought this up earlier about having a
> way
> > > to
> > > > > say
> > > > > > >> how
> > > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > > consumer
> > > > > > >> group.
> > > > > > >> > > If we can do this, then in case of mirroring applications
> we
> > > can
> > > > > do
> > > > > > >> this
> > > > > > >> > :
> > > > > > >> > > Lets say we have a mirroring application that consumes
> from
> > > > Kafka
> > > > > > >> cluster
> > > > > > >> > > A and produces to Kafka cluster B.
> > > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > > Kafka
> > > > > > >> service
> > > > > > >> > > providers can set a mirroring group saying that it will
> > take,
> > > > for
> > > > > > >> example
> > > > > > >> > > 300 consumer hosts/members to achieve the desired
> throughput
> > > and
> > > > > > >> latency
> > > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > > spare
> > > > > in
> > > > > > >> the
> > > > > > >> > > same group.
> > > > > > >> > > So when the first 300 members/consumers to join the group
> > will
> > > > > start
> > > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster
> B.
> > > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > > >> > > The moment one of the consumer (for example: consumer
> number
> > > 54)
> > > > > > from
> > > > > > >> the
> > > > > > >> > > first 300 members go out of the group (crossed session
> > > timeout),
> > > > > it
> > > > > > >> (the
> > > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> > the
> > > > > > >> consumer
> > > > > > >> > > member 54 to one of the spare hosts.
> > > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > > being a
> > > > > > >> part
> > > > > > >> > of
> > > > > > >> > > the spare pool.
> > > > > > >> > > This enables us to have lower session timeouts and low
> > latency
> > > > > > >> mirroring,
> > > > > > >> > > in cases where the service providers are OK with having
> > spare
> > > > > hosts.
> > > > > > >> > > This would mean that we would tolerate n consumer members
> > > > leaving
> > > > > > and
> > > > > > >> > > rejoining the group and still provide low latency as long
> > as n
> > > > <=
> > > > > > >> number
> > > > > > >> > of
> > > > > > >> > > spare consumers.
> > > > > > >> > > If there are no spare host available, we can get back to
> the
> > > > idea
> > > > > as
> > > > > > >> > > described in the KIP.
> > > > > > >> > Great idea! In fact on top of static membership we could
> later
> > > > > > introduce
> > > > > > >> > APIs to set hard-coded
> > > > > > >> > client ids to the group and replace the dead host, or as you
> > > > > proposed
> > > > > > to
> > > > > > >> > define spare host as
> > > > > > >> > what I understood as hot backup. I will put both Jason and
> > your
> > > > > > >> > suggestions into a separate section
> > > > > > >> > called "Future works". Note that this spare host idea may be
> > > also
> > > > > > >> solvable
> > > > > > >> > through rebalance protocol
> > > > > > >> > IMO.
> > > > > > >> >
> > > > > > >> > Thank you again for the great feedback!
> > > > > > >> >
> > > > > > >> > Boyang
> > > > > > >> > ________________________________
> > > > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > > > >> > To: dev@kafka.apache.org
> > > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > by
> > > > > > >> > specifying member id
> > > > > > >> >
> > > > > > >> > Hey Dong, sorry for missing your message. I couldn't find
> your
> > > > email
> > > > > > on
> > > > > > >> my
> > > > > > >> > thread, so I will just do a checklist here!
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > 1) The motivation currently explicitly states that the goal
> is
> > > to
> > > > > > >> improve
> > > > > > >> >
> > > > > > >> > performance for heavy state application. It seems that the
> > > > > motivation
> > > > > > >> can
> > > > > > >> >
> > > > > > >> > be stronger with the following use-case. Currently for
> > > MirrorMaker
> > > > > > >> cluster
> > > > > > >> >
> > > > > > >> > with e.g. 100 MirrorMaker processes, it will take a long
> time
> > to
> > > > > > rolling
> > > > > > >> >
> > > > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > process
> > > > > > restart
> > > > > > >> >
> > > > > > >> > will trigger a rebalance which currently pause the
> consumption
> > > of
> > > > > the
> > > > > > >> all
> > > > > > >> >
> > > > > > >> > partitions of the MirrorMaker cluster. With the change
> stated
> > in
> > > > > this
> > > > > > >> >
> > > > > > >> > patch, as long as a MirrorMaker can restart within the
> > specified
> > > > > > timeout
> > > > > > >> >
> > > > > > >> > (e.g. 2 minutes), then we only need constant number of
> > rebalance
> > > > > (e.g.
> > > > > > >> for
> > > > > > >> >
> > > > > > >> > leader restart) for the entire rolling bounce, which will
> > > > > > significantly
> > > > > > >> >
> > > > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > > > opinion,
> > > > > > >> the
> > > > > > >> >
> > > > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if
> > the
> > > > > > >> consumer
> > > > > > >> >
> > > > > > >> > process can be restarted within soon, which helps
> performance
> > > even
> > > > > if
> > > > > > >> >
> > > > > > >> > overhead of state shuffling for a given process is small.
> > > > > > >> >
> > > > > > >> > I just rephrased this part and added it to the KIP. Thanks
> for
> > > > > making
> > > > > > >> the
> > > > > > >> > motivation more solid!
> > > > > > >> >
> > > > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > > > writeup
> > > > > > >> style
> > > > > > >> > of other KIP (e.g. KIP-98) and list the interface change
> such
> > as
> > > > new
> > > > > > >> > configs (e.g. registration timeout), new request/response,
> new
> > > > > > >> AdminClient
> > > > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > Currently
> > > > > some
> > > > > > of
> > > > > > >> > these are specified in the Proposed Change section which
> makes
> > > it
> > > > a
> > > > > > bit
> > > > > > >> > inconvenient to understand the new interface that will be
> > > exposed
> > > > to
> > > > > > >> user.
> > > > > > >> > Explanation of the current two-phase rebalance protocol
> > probably
> > > > can
> > > > > > be
> > > > > > >> > moved out of public interface section.
> > > > > > >> > This is a great suggestion! I just consolidated all the
> public
> > > API
> > > > > > >> > changes, and the whole KIP
> > > > > > >> > looks much more organized!
> > > > > > >> >
> > > > > > >> > 3) There are currently two version of JoinGroupRequest in
> the
> > > KIP
> > > > > and
> > > > > > >> only
> > > > > > >> > one of them has field memberId. This seems confusing.
> > > > > > >> > Yep, I already found this issue and fixed it.
> > > > > > >> >
> > > > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > > > rebalance
> > > > > > >> could
> > > > > > >> > be helpful here, but we will make a call once we finished
> the
> > > > major
> > > > > > >> > implementation". So this seems to be still an open question
> in
> > > the
> > > > > > >> current
> > > > > > >> > design. We probably want to agree on this before voting for
> > the
> > > > KIP.
> > > > > > >> > We have finalized the idea that this API is needed.
> > > > > > >> >
> > > > > > >> > 5) The KIP currently adds new config MEMBER_NAME for
> consumer.
> > > Can
> > > > > you
> > > > > > >> > specify the name of the config key and the default config
> > value?
> > > > > > >> Possible
> > > > > > >> > default values include empty string or null (similar to
> > > > > > transaction.id<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=6Z0ODRuDX0I6g1QsYY54M%2BBypx3tuJ8fIvPSOLCBvcM%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > in
> > > > > > >> > producer config).
> > > > > > >> > I have defined the `member.name` in "New configuration"
> > > section.
> > > > > > >> >
> > > > > > >> > 6) Regarding the use of the topic "static_member_map" to
> > persist
> > > > > > member
> > > > > > >> > name map, currently if consumer coordinator broker goes
> > offline,
> > > > > > >> rebalance
> > > > > > >> > is triggered and consumers will try connect to the new
> > > > coordinator.
> > > > > If
> > > > > > >> > these consumers can connect to the new coordinator within
> > > > > > >> > max.poll.interval.ms<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=pwFb%2BRlHT3zteC318DrffUnPNCgucXIcFnsdtEl22BE%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > which by default is 5 minutes, given that broker can
> > > > > > >> > use a deterministic algorithm to determine the partition ->
> > > > > > member_name
> > > > > > >> > mapping, each consumer should get assigned the same set of
> > > > > partitions
> > > > > > >> > without requiring state shuffling. So it is not clear
> whether
> > we
> > > > > have
> > > > > > a
> > > > > > >> > strong use-case for this new logic. Can you help clarify
> what
> > is
> > > > the
> > > > > > >> > benefit of using topic "static_member_map" to persist member
> > > name
> > > > > map?
> > > > > > >> > I have discussed with Guozhang offline, and I believe
> reusing
> > > the
> > > > > > >> current
> > > > > > >> > `_consumer_offsets`
> > > > > > >> > topic is a better and unified solution.
> > > > > > >> >
> > > > > > >> > 7) Regarding the introduction of the expensionTimeoutMs
> > config,
> > > it
> > > > > is
> > > > > > >> > mentioned that "we are using expansion timeout to replace
> > > > rebalance
> > > > > > >> > timeout, which is configured by max.poll.intervals from
> client
> > > > side,
> > > > > > and
> > > > > > >> > using registration timeout to replace session timeout".
> > > Currently
> > > > > the
> > > > > > >> > default max.poll.interval.ms<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=pwFb%2BRlHT3zteC318DrffUnPNCgucXIcFnsdtEl22BE%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > is configured to be 5 minutes and there will
> > > > > > >> > be only one rebalance if all new consumers can join within 5
> > > > > minutes.
> > > > > > >> So it
> > > > > > >> > is not clear whether we have a strong use-case for this new
> > > > config.
> > > > > > Can
> > > > > > >> you
> > > > > > >> > explain what is the benefit of introducing this new config?
> > > > > > >> > Previously our goal is to use expansion timeout as a
> > workaround
> > > > for
> > > > > > >> > triggering multiple
> > > > > > >> > rebalances when scaling up members are not joining at the
> same
> > > > time.
> > > > > > It
> > > > > > >> is
> > > > > > >> > decided to
> > > > > > >> > be addressed by client side protocol change, so we will not
> > > > > introduce
> > > > > > >> > expansion timeout.
> > > > > > >> >
> > > > > > >> > 8) It is mentioned that "To distinguish between previous
> > version
> > > > of
> > > > > > >> > protocol, we will also increase the join group request
> version
> > > to
> > > > v4
> > > > > > >> when
> > > > > > >> > MEMBER_NAME is set" and "If the broker version is not the
> > latest
> > > > (<
> > > > > > v4),
> > > > > > >> > the join group request shall be downgraded to v3 without
> > setting
> > > > the
> > > > > > >> member
> > > > > > >> > Id". It is probably simpler to just say that this feature is
> > > > enabled
> > > > > > if
> > > > > > >> > JoinGroupRequest V4 is supported on both client and broker
> and
> > > > > > >> MEMBER_NAME
> > > > > > >> > is configured with non-empty string.
> > > > > > >> > Yep, addressed this!
> > > > > > >> >
> > > > > > >> > 9) It is mentioned that broker may return
> > > > NO_STATIC_MEMBER_INFO_SET
> > > > > > >> error
> > > > > > >> > in OffsetCommitResponse for "commit requests under static
> > > > > membership".
> > > > > > >> Can
> > > > > > >> > you clarify how broker determines whether the commit request
> > is
> > > > > under
> > > > > > >> > static membership?
> > > > > > >> >
> > > > > > >> > We have agreed that commit request shouldn't be affected by
> > the
> > > > new
> > > > > > >> > membership, thus
> > > > > > >> > removing it here. Thanks for catching this!
> > > > > > >> >
> > > > > > >> > Let me know if you have further suggestions or concerns.
> Thank
> > > you
> > > > > for
> > > > > > >> > your valuable feedback
> > > > > > >> > to help me design the KIP better! (And I will try to address
> > > your
> > > > > > >> > feedbacks in next round Mayuresh ??)
> > > > > > >> >
> > > > > > >> > Best,
> > > > > > >> > Boyang
> > > > > > >> > ________________________________
> > > > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > > > >> > To: dev@kafka.apache.org
> > > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > by
> > > > > > >> > specifying member id
> > > > > > >> >
> > > > > > >> > Hi Boyang,
> > > > > > >> >
> > > > > > >> > Thanks for updating the KIP. This is a step good direction
> for
> > > > > > stateful
> > > > > > >> > applications and also mirroring applications whose latency
> is
> > > > > affected
> > > > > > >> due
> > > > > > >> > to the rebalance issues that we have today.
> > > > > > >> >
> > > > > > >> > I had a few questions on the current version of the KIP :
> > > > > > >> > For the effectiveness of the KIP, consumer with member.name
> > set
> > > > > will
> > > > > > >> *not
> > > > > > >> > send leave group request* when they go offline
> > > > > > >> >
> > > > > > >> > > By this you mean, even if the application has not called
> > > > > > >> > > KafkaConsumer.poll() within session timeout, it will not
> be
> > > > > sending
> > > > > > >> the
> > > > > > >> > > LeaveGroup request, right?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > > > member.id
> > > > > > }
> > > > > > >> to
> > > > > > >> > track member uniqueness.
> > > > > > >> >
> > > > > > >> > > When is the member.name removed from this map?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Member.id must be set if the *member.name <
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=1zOe9ZqIHB7lvt8XJt2jtLvRaP75G3OamvtLRLjysyo%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > *is already
> > > > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > > > >> >
> > > > > > >> > > How is this case handled on the client side? What is the
> > > > > application
> > > > > > >> that
> > > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Session timeout is the timeout we will trigger rebalance
> when
> > a
> > > > > member
> > > > > > >> goes
> > > > > > >> > offline for too long (not sending heartbeat request). To
> make
> > > > static
> > > > > > >> > membership effective, we should increase the default max
> > session
> > > > > > >> timeout to
> > > > > > >> > 30 min so that end user could config it freely.
> > > > > > >> >
> > > > > > >> > > This would mean that it might take more time to detect
> > unowned
> > > > > topic
> > > > > > >> > > partitions and may cause delay for applications that
> perform
> > > > data
> > > > > > >> > mirroring
> > > > > > >> > > tasks. I discussed this with our sre and we have a
> > suggestion
> > > to
> > > > > > make
> > > > > > >> > here
> > > > > > >> > > as listed below separately.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Currently there is a config called *rebalance timeout* which
> > is
> > > > > > >> configured
> > > > > > >> > by consumer *max.poll.intervals*. The reason we set it to
> poll
> > > > > > interval
> > > > > > >> is
> > > > > > >> > because consumer could only send request within the call of
> > > poll()
> > > > > and
> > > > > > >> we
> > > > > > >> > want to wait sufficient time for the join group request.
> When
> > > > > reaching
> > > > > > >> > rebalance timeout, the group will move towards
> > > completingRebalance
> > > > > > stage
> > > > > > >> > and remove unjoined groups
> > > > > > >> >
> > > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Currently there is a config called *rebalance timeout* which
> > is
> > > > > > >> configured
> > > > > > >> > by consumer *max.poll.intervals*. The reason we set it to
> poll
> > > > > > interval
> > > > > > >> is
> > > > > > >> > because consumer could only send request within the call of
> > > poll()
> > > > > and
> > > > > > >> we
> > > > > > >> > want to wait sufficient time for the join group request.
> When
> > > > > reaching
> > > > > > >> > rebalance timeout, the group will move towards
> > > completingRebalance
> > > > > > stage
> > > > > > >> > and remove unjoined groups. This is actually conflicting
> with
> > > the
> > > > > > >> design of
> > > > > > >> > static membership, because those temporarily unavailable
> > members
> > > > > will
> > > > > > >> > potentially reattempt the join group and trigger extra
> > > rebalances.
> > > > > > >> > Internally we would optimize this logic by having rebalance
> > > > timeout
> > > > > > >> only in
> > > > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > > > >> non-responsive
> > > > > > >> > members immediately.
> > > > > > >> >
> > > > > > >> > > What do you mean by " Internally we would optimize this
> > logic
> > > by
> > > > > > >> having
> > > > > > >> > > rebalance timeout only in charge of stopping prepare
> > rebalance
> > > > > > stage,
> > > > > > >> > > without removing non-responsive members immediately."
> There
> > > > would
> > > > > > not
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > > request
> > > > > > later,
> > > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Scale Up
> > > > > > >> >
> > > > > > >> > > The KIP talks about scale up scenario but its not quite
> > clear
> > > > how
> > > > > we
> > > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> > we
> > > > > adding
> > > > > > >> > status
> > > > > > >> > > "learner" ?. Can you shed more light on how this is
> handled
> > in
> > > > the
> > > > > > >> KIP,
> > > > > > >> > if
> > > > > > >> > > its handled?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > *Discussion*
> > > > > > >> > Larger session timeouts causing latency rise for getting
> data
> > > for
> > > > > > >> un-owned
> > > > > > >> > topic partitions :
> > > > > > >> >
> > > > > > >> > > I think Jason had brought this up earlier about having a
> way
> > > to
> > > > > say
> > > > > > >> how
> > > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > > consumer
> > > > > > >> group.
> > > > > > >> > > If we can do this, then in case of mirroring applications
> we
> > > can
> > > > > do
> > > > > > >> this
> > > > > > >> > :
> > > > > > >> > > Lets say we have a mirroring application that consumes
> from
> > > > Kafka
> > > > > > >> cluster
> > > > > > >> > > A and produces to Kafka cluster B.
> > > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > > Kafka
> > > > > > >> service
> > > > > > >> > > providers can set a mirroring group saying that it will
> > take,
> > > > for
> > > > > > >> example
> > > > > > >> > > 300 consumer hosts/members to achieve the desired
> throughput
> > > and
> > > > > > >> latency
> > > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > > spare
> > > > > in
> > > > > > >> the
> > > > > > >> > > same group.
> > > > > > >> > > So when the first 300 members/consumers to join the group
> > will
> > > > > start
> > > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster
> B.
> > > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > > >> > > The moment one of the consumer (for example: consumer
> number
> > > 54)
> > > > > > from
> > > > > > >> the
> > > > > > >> > > first 300 members go out of the group (crossed session
> > > timeout),
> > > > > it
> > > > > > >> (the
> > > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> > the
> > > > > > >> consumer
> > > > > > >> > > member 54 to one of the spare hosts.
> > > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > > being a
> > > > > > >> part
> > > > > > >> > of
> > > > > > >> > > the spare pool.
> > > > > > >> > > This enables us to have lower session timeouts and low
> > latency
> > > > > > >> mirroring,
> > > > > > >> > > in cases where the service providers are OK with having
> > spare
> > > > > hosts.
> > > > > > >> > > This would mean that we would tolerate n consumer members
> > > > leaving
> > > > > > and
> > > > > > >> > > rejoining the group and still provide low latency as long
> > as n
> > > > <=
> > > > > > >> number
> > > > > > >> > of
> > > > > > >> > > spare consumers.
> > > > > > >> > > If there are no spare host available, we can get back to
> the
> > > > idea
> > > > > as
> > > > > > >> > > described in the KIP.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> >
> > > > > > >> > Mayuresh
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > > > >> > konstantine@confluent.io> wrote:
> > > > > > >> >
> > > > > > >> > > Hi Boyang.
> > > > > > >> > >
> > > > > > >> > > Thanks for preparing this KIP! It is making good progress
> > and
> > > > will
> > > > > > be
> > > > > > >> a
> > > > > > >> > > great improvement for stateful Kafka applications.
> > > > > > >> > >
> > > > > > >> > > Apologies for my late reply, I was away for a while. Lots
> of
> > > > great
> > > > > > >> > comments
> > > > > > >> > > so far, so I'll probably second most of them in what I
> > suggest
> > > > > below
> > > > > > >> at
> > > > > > >> > > this point.
> > > > > > >> > >
> > > > > > >> > > When I first read the KIP, I wanted to start at the end
> with
> > > > > > something
> > > > > > >> > that
> > > > > > >> > > wasn't highlighted a lot. That was the topic related to
> > > handling
> > > > > > >> > duplicate
> > > > > > >> > > members. I see now that the initial suggestion of handling
> > > this
> > > > > > >> situation
> > > > > > >> > > during offset commit has been removed, and I agree with
> > that.
> > > > > Issues
> > > > > > >> > > related to membership seem to be handled better when the
> > > member
> > > > > > joins
> > > > > > >> the
> > > > > > >> > > group rather than when it tries to commit offsets. This
> also
> > > > > > >> simplifies
> > > > > > >> > how
> > > > > > >> > > many request types need to change in order to incorporate
> > the
> > > > new
> > > > > > >> member
> > > > > > >> > > name field.
> > > > > > >> > >
> > > > > > >> > > I also agree with what Jason and Guozhang have said
> > regarding
> > > > > > >> timeouts.
> > > > > > >> > > Although semantically, it's easier to think of every
> > operation
> > > > > > having
> > > > > > >> its
> > > > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > > > >> consolidation
> > > > > > >> > > seems preferable here. The definition of embedded
> protocols
> > on
> > > > top
> > > > > > of
> > > > > > >> the
> > > > > > >> > > base group membership protocol for rebalancing gives
> enough
> > > > > > >> flexibility
> > > > > > >> > to
> > > > > > >> > > address such needs in each client component separately.
> > > > > > >> > >
> > > > > > >> > > Finally, some minor comments:
> > > > > > >> > > In a few places the new/proposed changes are referred to
> as
> > > > > > "current".
> > > > > > >> > > Which is a bit confusing considering that there is a
> > protocol
> > > in
> > > > > > place
> > > > > > >> > > already, and by "current" someone might understand the
> > > existing
> > > > > one.
> > > > > > >> I'd
> > > > > > >> > > recommend using new/proposed or equivalent when referring
> to
> > > > > changes
> > > > > > >> > > introduced with KIP-345 and current/existing or equivalent
> > > when
> > > > > > >> referring
> > > > > > >> > > to existing behavior.
> > > > > > >> > >
> > > > > > >> > > There's the following sentence in the "Public Interfaces"
> > > > section:
> > > > > > >> > > "Since for many stateful consumer/stream applications, the
> > > state
> > > > > > >> > shuffling
> > > > > > >> > > is more painful than short time partial unavailability."
> > > > > > >> > > However, my understanding is that the changes proposed
> with
> > > > > KIP-345
> > > > > > >> will
> > > > > > >> > > not exploit any partial availability. A suggestion for
> > dealing
> > > > > with
> > > > > > >> > > temporary imbalances has been made in "Incremental
> > Cooperative
> > > > > > >> > Rebalancing"
> > > > > > >> > > which can work well with KIP-345, but here I don't see
> > > proposed
> > > > > > >> changes
> > > > > > >> > > that suggest that some resources (e.g. partitions) will
> keep
> > > > being
> > > > > > >> used
> > > > > > >> > > while others will not be utilized. Thus, you might want to
> > > > adjust
> > > > > > this
> > > > > > >> > > sentence. Correct me if I'm missing something related to
> > that.
> > > > > > >> > >
> > > > > > >> > > In the rejected alternatives, under point 2) I read "we
> can
> > > copy
> > > > > the
> > > > > > >> > member
> > > > > > >> > > id to the config files". I believe it means to say "member
> > > name"
> > > > > > >> unless
> > > > > > >> > I'm
> > > > > > >> > > missing something about reusing member ids. Also below I
> > read:
> > > > "By
> > > > > > >> > allowing
> > > > > > >> > > consumers to optionally specifying a member id" which
> > probably
> > > > > > implies
> > > > > > >> > > "member name" again. In a sense this section highlights a
> > > > > potential
> > > > > > >> > > confusion between member name and member id. I wonder if
> we
> > > > could
> > > > > > >> come up
> > > > > > >> > > with a better term for the new field. StaticTag,
> > StaticLabel,
> > > or
> > > > > > even
> > > > > > >> > > StaticName are some suggestions that could potentially
> help
> > > with
> > > > > > >> > confusion
> > > > > > >> > > between MemberId and MemberName and what corresponds to
> > what.
> > > > But
> > > > > I
> > > > > > >> > > wouldn't like to disrupt the discussion with naming
> > > conventions
> > > > > too
> > > > > > >> much
> > > > > > >> > at
> > > > > > >> > > this point. I just mention it here as a thought.
> > > > > > >> > >
> > > > > > >> > > Looking forward to see the final details of this KIP.
> Great
> > > work
> > > > > so
> > > > > > >> far!
> > > > > > >> > >
> > > > > > >> > > Konstantine
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > > > bchen11@outlook.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Thanks Guozhang for the great summary here, and I have
> > been
> > > > > > >> following
> > > > > > >> > up
> > > > > > >> > > > the action items here.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > > > timeout
> > > > > > and
> > > > > > >> > > > registration timeout. Great to see them being addressed
> in
> > > > > client
> > > > > > >> side!
> > > > > > >> > > >   2.  I double checked the design and I believe that it
> is
> > > ok
> > > > to
> > > > > > >> have
> > > > > > >> > > both
> > > > > > >> > > > static member and dynamic member co-exist in the same
> > group.
> > > > So
> > > > > > the
> > > > > > >> > > upgrade
> > > > > > >> > > > shouldn't be destructive and we are removing the two
> > > > membership
> > > > > > >> > protocol
> > > > > > >> > > > switching APIs.
> > > > > > >> > > >   3.  I only have question about this one. I'm still
> > reading
> > > > the
> > > > > > >> > > KafkaApis
> > > > > > >> > > > code here. Should I just use the same authorization
> logic
> > > for
> > > > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > > > >> > > >   4.  I'm very excited to see this work with K8! Like
> you
> > > > > > suggested,
> > > > > > >> > this
> > > > > > >> > > > feature could be better addressed in a separate KIP
> > because
> > > it
> > > > > is
> > > > > > >> > pretty
> > > > > > >> > > > independent. I could start drafting the KIP once the
> > current
> > > > > > >> proposal
> > > > > > >> > is
> > > > > > >> > > > approved.
> > > > > > >> > > >   5.  I believe that we don't need fencing in offset
> > commit
> > > > > > request,
> > > > > > >> > > since
> > > > > > >> > > > duplicate member.name issue could be handled by join
> > group
> > > > > > >> request. We
> > > > > > >> > > > shall reject join group with known member name but no
> > member
> > > > id
> > > > > > >> (which
> > > > > > >> > > > means we already have an active member using this
> > identity).
> > > > > > >> > > >   6.  I agree to remove that internal config once we
> move
> > > > > forward
> > > > > > >> with
> > > > > > >> > > > static membership. And I already removed the entire
> > section
> > > > from
> > > > > > the
> > > > > > >> > KIP.
> > > > > > >> > > >
> > > > > > >> > > > Let me know if you have other concerns.
> > > > > > >> > > >
> > > > > > >> > > > Best,
> > > > > > >> > > > Boyang
> > > > > > >> > > > ________________________________
> > > > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > > > >> > > > To: dev
> > > > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > > > rebalances
> > > > > > >> by
> > > > > > >> > > > specifying member id
> > > > > > >> > > >
> > > > > > >> > > > Hello Boyang,
> > > > > > >> > > >
> > > > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > > > appreciate
> > > > > > >> your
> > > > > > >> > > > patience answering to the feedbacks from the community.
> > I'd
> > > > like
> > > > > > to
> > > > > > >> add
> > > > > > >> > > my
> > > > > > >> > > > 2cents here:
> > > > > > >> > > >
> > > > > > >> > > > 1. By introducing another two timeout configs,
> > > > > > registration_timeout
> > > > > > >> and
> > > > > > >> > > > expansion_timeout, we are effectively having four
> timeout
> > > > > configs:
> > > > > > >> > > session
> > > > > > >> > > > timeout, rebalance timeout (configured as "
> > > > max.poll.interval.ms
> > > > > "
> > > > > > on
> > > > > > >> > > client
> > > > > > >> > > > side), and these two. Interplaying these timeout configs
> > can
> > > > be
> > > > > > >> quite
> > > > > > >> > > hard
> > > > > > >> > > > for users with such complexity, and hence I'm wondering
> if
> > > we
> > > > > can
> > > > > > >> > > simplify
> > > > > > >> > > > the situation with as less possible timeout configs as
> > > > possible.
> > > > > > >> Here
> > > > > > >> > is
> > > > > > >> > > a
> > > > > > >> > > > concrete suggestion I'd like propose:
> > > > > > >> > > >
> > > > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> > > addition
> > > > > to
> > > > > > >> the
> > > > > > >> > > > session_timeout for static members, we can just reuse
> the
> > > > > > >> > session_timeout
> > > > > > >> > > > and ask users to set it to a larger value when they are
> > > > > upgrading
> > > > > > a
> > > > > > >> > > dynamic
> > > > > > >> > > > client to a static client by setting the "member.name"
> at
> > > the
> > > > > > same
> > > > > > >> > time.
> > > > > > >> > > > By
> > > > > > >> > > > default, the broker-side min.session.timeout is 6
> seconds
> > > and
> > > > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable
> > to
> > > me
> > > > > (we
> > > > > > >> can
> > > > > > >> > of
> > > > > > >> > > > course modify this broker config to enlarge the valid
> > > interval
> > > > > if
> > > > > > we
> > > > > > >> > want
> > > > > > >> > > > in practice). And then we should also consider removing
> > the
> > > > > > >> condition
> > > > > > >> > for
> > > > > > >> > > > marking a client as failed if the rebalance timeout has
> > > > reached
> > > > > > >> while
> > > > > > >> > the
> > > > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > > > session_timeout
> > > > > > >> > and
> > > > > > >> > > > rebalance_timeout are totally separated: the former is
> > only
> > > > used
> > > > > > to
> > > > > > >> > > > determine if a consumer member of the group should be
> > marked
> > > > as
> > > > > > >> failed
> > > > > > >> > > and
> > > > > > >> > > > kicked out of the group, and the latter is only used to
> > > > > determine
> > > > > > >> the
> > > > > > >> > > > longest time coordinator should wait for
> PREPARE_REBALANCE
> > > > > phase.
> > > > > > In
> > > > > > >> > > other
> > > > > > >> > > > words if a member did not send the JoinGroup in time of
> > the
> > > > > > >> > > > rebalance_timeout, we still include it in the new
> > generation
> > > > of
> > > > > > the
> > > > > > >> > group
> > > > > > >> > > > and use its old subscription info to send to leader for
> > > > > > assignment.
> > > > > > >> > Later
> > > > > > >> > > > if the member came back with HeartBeat request, we can
> > still
> > > > > > follow
> > > > > > >> the
> > > > > > >> > > > normal path to bring it to the latest generation while
> > > > checking
> > > > > > that
> > > > > > >> > its
> > > > > > >> > > > sent JoinGroup request contains the same subscription
> info
> > > as
> > > > we
> > > > > > >> used
> > > > > > >> > to
> > > > > > >> > > > assign the partitions previously (which should be likely
> > the
> > > > > case
> > > > > > in
> > > > > > >> > > > practice). In addition, we should let static members to
> > not
> > > > send
> > > > > > the
> > > > > > >> > > > LeaveGroup request when it is gracefully shutdown, so
> > that a
> > > > > > static
> > > > > > >> > > member
> > > > > > >> > > > can only be leaving the group if its session has timed
> > out,
> > > OR
> > > > > it
> > > > > > >> has
> > > > > > >> > > been
> > > > > > >> > > > indicated to not exist in the group any more (details
> > > below).
> > > > > > >> > > >
> > > > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > > > Cooperative
> > > > > > >> > > > Rebalancing, in which we will encode the "when to
> > rebalance"
> > > > > logic
> > > > > > >> at
> > > > > > >> > the
> > > > > > >> > > > application level, instead of at the protocol level. By
> > > doing
> > > > > this
> > > > > > >> we
> > > > > > >> > can
> > > > > > >> > > > also enable a few other optimizations, e.g. at the
> Streams
> > > > level
> > > > > > to
> > > > > > >> > first
> > > > > > >> > > > build up the state store as standby tasks and then
> > trigger a
> > > > > > second
> > > > > > >> > > > rebalance to actually migrate the active tasks while
> > keeping
> > > > the
> > > > > > >> actual
> > > > > > >> > > > rebalance latency and hence unavailability window to be
> > > small
> > > > (
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=5T%2FocdSTFR0kDWba3yGPs0tZNRJPCSnKBWrkiywFlwI%3D&amp;reserved=0
> > > > > > >> > > ).
> > > > > > >> > > > I'd propose we align
> > > > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > > > >> > expansion_timeout
> > > > > > >> > > as
> > > > > > >> > > > part of the protocol layer, but only do that at the
> > > > > application's
> > > > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We
> > can
> > > > > > still,
> > > > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > > > >> > > > <
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C99d1b807ce7e4fd280ac08d65718dcda%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636792161627905154&amp;sdata=HBnPB8ua6UcXzf58FZIdUn%2Flsy%2BhTgjF80OjtNjFIl0%3D&amp;reserved=0
> > > > > > >> > > >*"
> > > > > > >> > > > though as part of this KIP
> > > > > > >> > > > since we have discussed about its limit and think it is
> > > > actually
> > > > > > >> not a
> > > > > > >> > > very
> > > > > > >> > > > good design and could be replaced with client-side logic
> > > > above.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> > > this
> > > > > KIP.
> > > > > > >> More
> > > > > > >> > > > specifically, let's say after we have upgraded broker
> > > version
> > > > to
> > > > > > be
> > > > > > >> > able
> > > > > > >> > > to
> > > > > > >> > > > recognize the new versions of JoinGroup request and the
> > > admin
> > > > > > >> requests,
> > > > > > >> > > how
> > > > > > >> > > > should we upgrade the clients and enable static groups?
> On
> > > top
> > > > > of
> > > > > > my
> > > > > > >> > head
> > > > > > >> > > > if we do a rolling bounce in which we set the
> member.name
> > > > > config
> > > > > > as
> > > > > > >> > well
> > > > > > >> > > > as
> > > > > > >> > > > optionally increase the session.timeout config when we
> > > bounce
> > > > > each
> > > > > > >> > > > instance, then during this rolling bounces we will have
> a
> > > > group
> > > > > > >> > contained
> > > > > > >> > > > with both dynamic members and static members. It means
> > that
> > > we
> > > > > > >> should
> > > > > > >> > > have
> > > > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > > > JoinGroup
> > > > > > >> > > requests
> > > > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > > > member.id"
> > > > > > >> > > mapping
> > > > > > >> > > > will only be partial at this scenario. Also could you
> > > describe
> > > > > if
> > > > > > >> the
> > > > > > >> > > > upgrade to the first version that support this feature
> > would
> > > > > ever
> > > > > > >> get
> > > > > > >> > any
> > > > > > >> > > > benefits, or only the future upgrade path for rolling
> > > bounces
> > > > > > could
> > > > > > >> get
> > > > > > >> > > > benefits out of this feature?
> > > > > > >> > > >
> > > > > > >> > > > If that's the case and we will do 1) as suggested above,
> > do
> > > we
> > > > > > still
> > > > > > >> > need
> > > > > > >> > > > the enableStaticMembership and enableDynamicMembership
> > admin
> > > > > > >> requests
> > > > > > >> > any
> > > > > > >> > > > more? Seems it is not necessary any more as we will only
> > > have
> > > > > the
> > > > > > >> > notion
> > > > > > >> > > of
> > > > > > >> > > > "dynamic or static members" that can co-exist in a group
> > > while
> > > > > > >> there no
> > > > > > >> > > > notion of "dynamic or static groups", and hence these
> two
> > > > > requests
> > > > > > >> are
> > > > > > >> > > not
> > > > > > >> > > > needed anymore.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 3. We need to briefly talk about the implications for
> ACL
> > as
> > > > we
> > > > > > >> > introduce
> > > > > > >> > > > new admin requests that are related to a specific
> > group.id.
> > > > For
> > > > > > >> > example,
> > > > > > >> > > > we
> > > > > > >> > > > need to make sure that whoever created the group or
> joined
> > > the
> > > > > > group
> > > > > > >> > can
> > > > > > >> > > > actually send admin requests for the group, otherwise
> the
> > > > > > >> application
> > > > > > >> > > > owners need to bother the Kafka operators on a
> > multi-tenant
> > > > > > cluster
> > > > > > >> > every
> > > > > > >> > > > time they want to send any admin requests for their
> groups
> > > > which
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > > > an operational nightmare.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 4. I like Jason's suggestion of adding an optional field
> > for
> > > > the
> > > > > > >> list
> > > > > > >> > of
> > > > > > >> > > > member names, and I'm wondering if that can be done as
> > part
> > > of
> > > > > the
> > > > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > > > members,
> > > > > > we
> > > > > > >> > will
> > > > > > >> > > > enforce a rebalance immediately since it indicates that
> > some
> > > > > > static
> > > > > > >> > > member
> > > > > > >> > > > will be officially kicked out of the group and some new
> > > static
> > > > > > >> members
> > > > > > >> > > may
> > > > > > >> > > > be added. So back to 1.a) above, a static member can
> only
> > be
> > > > > > kicked
> > > > > > >> out
> > > > > > >> > > of
> > > > > > >> > > > the group if a) its session (arguably long period of
> time)
> > > has
> > > > > > timed
> > > > > > >> > out,
> > > > > > >> > > > and b) this admin request explicitly state that it is no
> > > > longer
> > > > > > >> part of
> > > > > > >> > > the
> > > > > > >> > > > group. As for execution I'm fine with keeping it as a
> > future
> > > > > work
> > > > > > of
> > > > > > >> > this
> > > > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > > > >> > > >
> > > > > > >> > > > Following are minor comments:
> > > > > > >> > > >
> > > > > > >> > > > 5. I'm not sure if we need to include "member.name" as
> > part
> > > > of
> > > > > > the
> > > > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > > > memberId
> > > > > > >> plus
> > > > > > >> > > the
> > > > > > >> > > > generation number should be sufficient for fencing even
> > with
> > > > > > static
> > > > > > >> > > > members.
> > > > > > >> > > >
> > > > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get
> rid
> > > of
> > > > > the
> > > > > > "
> > > > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > Guozhang
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Hey Boyang,
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks for the proposal! This is very useful. I have
> > some
> > > > > > comments
> > > > > > >> > > below:
> > > > > > >> > > > >
> > > > > > >> > > > > 1) The motivation currently explicitly states that the
> > > goal
> > > > is
> > > > > > to
> > > > > > >> > > improve
> > > > > > >> > > > > performance for heavy state application. It seems that
> > the
> > > > > > >> motivation
> > > > > > >> > > can
> > > > > > >> > > > > be stronger with the following use-case. Currently for
> > > > > > MirrorMaker
> > > > > > >> > > > cluster
> > > > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a
> long
> > > > time
> > > > > to
> > > > > > >> > > rolling
> > > > > > >> > > > > bounce the entire MirrorMaker cluster. Each
> MirrorMaker
> > > > > process
> > > > > > >> > restart
> > > > > > >> > > > > will trigger a rebalance which currently pause the
> > > > consumption
> > > > > > of
> > > > > > >> the
> > > > > > >> > > all
> > > > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > > > stated
> > > > > in
> > > > > > >> this
> > > > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > > > specified
> > > > > > >> > > timeout
> > > > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > > > rebalance
> > > > > > >> > (e.g.
> > > > > > >> > > > for
> > > > > > >> > > > > leader restart) for the entire rolling bounce, which
> > will
> > > > > > >> > significantly
> > > > > > >> > > > > improves the availability of the MirrorMaker pipeline.
> > In
> > > my
> > > > > > >> opinion,
> > > > > > >> > > the
> > > > > > >> > > > > main benefit of the KIP is to avoid unnecessary
> > rebalance
> > > if
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > > process can be restarted within soon, which helps
> > > > performance
> > > > > > >> even if
> > > > > > >> > > > > overhead of state shuffling for a given process is
> > small.
> > > > > > >> > > > >
> > > > > > >> > > > > 2) In order to simplify the KIP reading, can you
> follow
> > > the
> > > > > > >> writeup
> > > > > > >> > > style
> > > > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface
> change
> > > > such
> > > > > as
> > > > > > >> new
> > > > > > >> > > > > configs (e.g. registration timeout), new
> > request/response,
> > > > new
> > > > > > >> > > > AdminClient
> > > > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > > > Currently
> > > > > > >> some
> > > > > > >> > > of
> > > > > > >> > > > > these are specified in the Proposed Change section
> which
> > > > makes
> > > > > > it
> > > > > > >> a
> > > > > > >> > bit
> > > > > > >> > > > > inconvenient to understand the new interface that will
> > be
> > > > > > exposed
> > > > > > >> to
> > > > > > >> > > > user.
> > > > > > >> > > > > Explanation of the current two-phase rebalance
> protocol
> > > > > probably
> > > > > > >> can
> > > > > > >> > be
> > > > > > >> > > > > moved out of public interface section.
> > > > > > >> > > > >
> > > > > > >> > > > > 3) There are currently two version of JoinGroupRequest
> > in
> > > > the
> > > > > > KIP
> > > > > > >> and
> > > > > > >> > > > only
> > > > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > > > >> > > > >
> > > > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to
> > force
> > > > > > >> rebalance
> > > > > > >> > > could
> > > > > > >> > > > > be helpful here, but we will make a call once we
> > finished
> > > > the
> > > > > > >> major
> > > > > > >> > > > > implementation". So this seems to be still an open
> > > question
> > > > in
> > > > > > the
> > > > > > >> > > > current
> > > > > > >> > > > > design. We probably want to agree on this before
> voting
> > > for
> > > > > the
> > > > > > >> KIP.
> > > > > > >> > > > >
> > > > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > > > consumer.
> > > > > > Can
> > > > > > >> > you
> > > > > > >> > > > > specify the name of the config key and the default
> > config
> > > > > value?
> > > > > > >> > > Possible
> > > > > > >> > > > > default values include empty string or null (similar
> to
> > > > > > >> > transaction.id
> > > > > > >> > > > in
> > > > > > >> > > > > producer config).
> > > > > > >> > > > >
> > > > > > >> > > > > 6) Regarding the use of the topic "static_member_map"
> to
> > > > > persist
> > > > > > >> > member
> > > > > > >> > > > > name map, currently if consumer coordinator broker
> goes
> > > > > offline,
> > > > > > >> > > > rebalance
> > > > > > >> > > > > is triggered and consumers will try connect to the new
> > > > > > >> coordinator.
> > > > > > >> > If
> > > > > > >> > > > > these consumers can connect to the new coordinator
> > within
> > > > > > >> > > > > max.poll.interval.ms which by default is 5 minutes,
> > given
> > > > > that
> > > > > > >> > broker
> > > > > > >> > > > can
> > > > > > >> > > > > use a deterministic algorithm to determine the
> partition
> > > ->
> > > > > > >> > member_name
> > > > > > >> > > > > mapping, each consumer should get assigned the same
> set
> > of
> > > > > > >> partitions
> > > > > > >> > > > > without requiring state shuffling. So it is not clear
> > > > whether
> > > > > we
> > > > > > >> > have a
> > > > > > >> > > > > strong use-case for this new logic. Can you help
> clarify
> > > > what
> > > > > is
> > > > > > >> the
> > > > > > >> > > > > benefit of using topic "static_member_map" to persist
> > > member
> > > > > > name
> > > > > > >> > map?
> > > > > > >> > > > >
> > > > > > >> > > > > 7) Regarding the introduction of the
> expensionTimeoutMs
> > > > > config,
> > > > > > >> it is
> > > > > > >> > > > > mentioned that "we are using expansion timeout to
> > replace
> > > > > > >> rebalance
> > > > > > >> > > > > timeout, which is configured by max.poll.intervals
> from
> > > > client
> > > > > > >> side,
> > > > > > >> > > and
> > > > > > >> > > > > using registration timeout to replace session
> timeout".
> > > > > > Currently
> > > > > > >> the
> > > > > > >> > > > > default max.poll.interval.ms is configured to be 5
> > > minutes
> > > > > and
> > > > > > >> there
> > > > > > >> > > > will
> > > > > > >> > > > > be only one rebalance if all new consumers can join
> > > within 5
> > > > > > >> minutes.
> > > > > > >> > > So
> > > > > > >> > > > it
> > > > > > >> > > > > is not clear whether we have a strong use-case for
> this
> > > new
> > > > > > >> config.
> > > > > > >> > Can
> > > > > > >> > > > you
> > > > > > >> > > > > explain what is the benefit of introducing this new
> > > config?
> > > > > > >> > > > >
> > > > > > >> > > > > 8) It is mentioned that "To distinguish between
> previous
> > > > > version
> > > > > > >> of
> > > > > > >> > > > > protocol, we will also increase the join group request
> > > > version
> > > > > > to
> > > > > > >> v4
> > > > > > >> > > when
> > > > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not
> > the
> > > > > latest
> > > > > > >> (<
> > > > > > >> > > v4),
> > > > > > >> > > > > the join group request shall be downgraded to v3
> without
> > > > > setting
> > > > > > >> the
> > > > > > >> > > > member
> > > > > > >> > > > > Id". It is probably simpler to just say that this
> > feature
> > > is
> > > > > > >> enabled
> > > > > > >> > if
> > > > > > >> > > > > JoinGroupRequest V4 is supported on both client and
> > broker
> > > > and
> > > > > > >> > > > MEMBER_NAME
> > > > > > >> > > > > is configured with non-empty string.
> > > > > > >> > > > >
> > > > > > >> > > > > 9) It is mentioned that broker may return
> > > > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > > > >> > > error
> > > > > > >> > > > > in OffsetCommitResponse for "commit requests under
> > static
> > > > > > >> > membership".
> > > > > > >> > > > Can
> > > > > > >> > > > > you clarify how broker determines whether the commit
> > > request
> > > > > is
> > > > > > >> under
> > > > > > >> > > > > static membership?
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks,
> > > > > > >> > > > > Dong
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > --
> > > > > > >> > > > -- Guozhang
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > --
> > > > > > >> > -Regards,
> > > > > > >> > Mayuresh R. Gharat
> > > > > > >> > (862) 250-7125
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -Regards,
> > > > > > > Mayuresh R. Gharat
> > > > > > > (862) 250-7125
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -Regards,
> > > > > > Mayuresh R. Gharat
> > > > > > (862) 250-7125
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> > --
> > -- Guozhang
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


--
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Boyang,

For Streams, I think we do not need an extra config for the instance id,
instead, we can re-use the way we construct the embedded consumer's client
id as:

[streams client-id] + "-StreamThread-" + [thread-id] + "-consumer"

So as long as user's specify the unique streams client-id, the resulted
consumer client-id / instance-id should be unique as well already.

As for the LeaveGroupRequest, as I understand it, your concern is that when
we are shutting down a single Streams instance that may contain multiple
threads, shutting down that instance would mean shutting down multiple
members. Personally I'd prefer to make the LeaveGroupRequest API more
general and less inclined to Streams (I think Mayuresh also suggested
this). So I'd suggest that we keep the LeaveGroupRequest API as suggested,
i.e. a list of member.instance.ids. And in Streams we can add a new API in
KafkaStreams to expose:

1) the list of embedded consumer / producer client ids,
2) the producer's txn ids if EOS is turned on, and
3) the consumer's instance ids.

So that Streams operators can read those values from KafkaStreams directly
before shutting it down and use the list in the LeaveGroupRequest API. How
about that?


Guozhang


On Fri, Nov 30, 2018 at 7:45 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> I like Guozhang's suggestion to not have to wait for session timeout in
> case we know that we want to downsize the consumer group and redistribute
> the partitions among the remaining consumers.
> IIUC, with the above suggestions, the admin api
> "removeMemberFromGroup(groupId, list[instanceId])" or
> "removeMemberFromGroup(groupId, instanceId)", will automatically cause a
> rebalance, right?
> I would prefer ist[instanceid] because that's more general scenario.
>
> Also I was thinking if we can have a replace API, that takes in a map of
> old to new instance Ids. Such that we can replace a consumer.
> IF we have this api, and if a consumer host goes down due to hardware
> issues, we can have another host spin up and take its place. This is like a
> cold backup which can be a step towards providing the hot backup that we
> discussed earlier in the KIP.
> Thoughts?
>
> Thanks,
>
> Mayuresh
>
> On Thu, Nov 29, 2018 at 1:30 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > In fact I feel that it's more convenient for user to specify a list of
> > instance id prefixes. Because
> > for general consumer application we couldn't always find a proper prefix
> > to remove a list of consumers.
> > So we are either adding list[instanceid prefix], or we could add two
> > fields: instanceid prefix, and list[instanceid]
> > for clarity purpose. As you know, two options are equivalent since full
> > name is subset of prefix.
> >
> > Let me know your thoughts!
> >
> > Boyang
> > ________________________________
> > From: Boyang Chen <bc...@outlook.com>
> > Sent: Thursday, November 29, 2018 3:39 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Thanks Guozhang for the new proposal here!
> >
> > So I'd like to propose a slightly modified version of LeaveGroupRequest:
> > instead of letting the static member consumer client themselves to send
> the
> > request (which means we still need to have some hidden configs to turn it
> > off like we did today), how about just letting any other client to send
> > this request since the LeaveGroupRequest only requires group.id and
> > member.id? So back to your operational scenarios, if some static member
> > has
> > been found crashed and it is not likely to comeback, or we simply want to
> > shrink the size of the group by shutting down some static members, we can
> > use an admin client to send the LeaveGroupRequest after the instance has
> > been completely shutdown or crashed to kick them out of the group and
> also
> > triggers the rebalance.
> >
> > One issue though, is that users may not know the member id required in
> the
> > LeaveGroupRequest. To work around it we can add the `group.instance.id`
> > along with the member id as well and then allow member id null-able. The
> > coordinator logic would then be modified as 1) if member.id is
> specified,
> > ignore instance.id and always use member.id to find the member to kick
> > out,
> > 2) otherwise, try with the instance.id to find the corresponding
> member.id
> > and kick it out, 3) if none is found, reject with an error code.
> >
> > So in sum the alternative changes are:
> >
> > a) Modify LeaveGroupRequest to add group.instance.id
> > b) Modify coordinator logic to handle such request on the broker side.
> > c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> > instanceId)" which will be translated as a LeaveGroupRequest.
> > d) [Optional] we can even batch the request by allowing
> > "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> > member.id`
> > and `instance.id` field of LeaveGroupRequest to be an array instead of a
> > single entry.
> > e) We can also remove the admin ConsumerRebalanceRequest as well for
> > simplicity (why not? paranoid of having as less request protocols as
> > possible :), as it is not needed anymore with the above proposal.
> > I agree that reusing LeaveGroupRequest is actually a good idea: we only
> > need to iterate
> > over an existing request format. Also I found that we haven't discussed
> > how we want to enable
> > this feature on Streaming applications, which is different from common
> > consumer application in that
> > Stream app uses stream thread as individual consumer.
> > For example if user specifies the client id, the stream consumer client
> id
> > will be like:
> > User client id + "-StreamThread-" + thread id + "-consumer"
> >
> > So I'm thinking we should do sth similar for defining group.instance.id
> > on Stream. We shall define another
> > config called `stream.instance.id` which would be used as prefix, and
> for
> > each thread consumer the formula
> > will look like:
> > `group.instance.id` = `stream.instance.id` + "-" + thread id +
> "-consumer"
> >
> > And for the ease of use, the interface of leave group request could
> > include `group.instance.id.prefix` instead of
> > `group.instance.id` so that we could batch remove consumers relating to
> a
> > single stream instance. This is more intuitive
> > and flexible since specifying names of 16~32 * n (n = number of stream
> > instances to shut down) consumers is not an easy
> > job without client management tooling.
> >
> > How does this workaround sound?
> >
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Thursday, November 29, 2018 2:38 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > I was thinking that with the optional static members in the admin
> > ConsumerRebalanceRequest it should be sufficient to kick out the static
> > member before their session timeout (arguably long in practice) have not
> > reached. But now I see your concern is that in some situations the admin
> > operators may not even know the full list of static members, but ONLY
> know
> > which static member has failed and hence would like to kick out of the
> > group.
> >
> > So I'd like to propose a slightly modified version of LeaveGroupRequest:
> > instead of letting the static member consumer client themselves to send
> the
> > request (which means we still need to have some hidden configs to turn it
> > off like we did today), how about just letting any other client to send
> > this request since the LeaveGroupRequest only requires group.id and
> > member.id? So back to your operational scenarios, if some static member
> > has
> > been found crashed and it is not likely to comeback, or we simply want to
> > shrink the size of the group by shutting down some static members, we can
> > use an admin client to send the LeaveGroupRequest after the instance has
> > been completely shutdown or crashed to kick them out of the group and
> also
> > triggers the rebalance.
> >
> > One issue though, is that users may not know the member id required in
> the
> > LeaveGroupRequest. To work around it we can add the `group.instance.id`
> > along with the member id as well and then allow member id null-able. The
> > coordinator logic would then be modified as 1) if member.id is
> specified,
> > ignore instance.id and always use member.id to find the member to kick
> > out,
> > 2) otherwise, try with the instance.id to find the corresponding
> member.id
> > and kick it out, 3) if none is found, reject with an error code.
> >
> > So in sum the alternative changes are:
> >
> > a) Modify LeaveGroupRequest to add group.instance.id
> > b) Modify coordinator logic to handle such request on the broker side.
> > c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> > instanceId)" which will be translated as a LeaveGroupRequest.
> > d) [Optional] we can even batch the request by allowing
> > "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> > member.id`
> > and `instance.id` field of LeaveGroupRequest to be an array instead of a
> > single entry.
> > e) We can also remove the admin ConsumerRebalanceRequest as well for
> > simplicity (why not? paranoid of having as less request protocols as
> > possible :), as it is not needed anymore with the above proposal.
> >
> >
> > WDYT?
> >
> >
> > Guozhang
> >
> > On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Guozhang and Mayuresh for the follow up! Answers are listed
> below.
> > >
> > >
> > > >  5. Regarding "So in summary, *the member will only be removed due to
> > > > session timeout*. We shall remove it from both in-memory static
> member
> > > name
> > > > mapping and member list." If the rebalance is invoked manually using
> > the
> > > > the admin apis, how long should the group coordinator wait for the
> > > members
> > > > of the group to send a JoinGroupRequest for participating in the
> > > rebalance?
> > > > How is a lagging consumer handled?
> > >
> > > Great question. Let's use c1~c4 example here:
> > >
> > >   1.  Consumer c1, c2, c3, c4 in stable state
> > >   2.  c4 goes down and we detect this issue before session timeout
> > through
> > > client monitoring. Initiate a ConsumerRebalanceRequest.
> > >   3.  A rebalance will be kicking off, and after rebalance timeout we
> > > shall keep the same assignment for c1~4, if the session timeout for c4
> > > hasn't reached
> > >   4.  Group back to stable with c1~4 (although c4 is actually offline)
> > >   5.  c4 session timeout finally reached: another rebalance triggered.
> > >
> > > For step 3, if session timeout triggered within rebalance timeout, only
> > > c1~3 will be participating in the rebalance. This is what we mean by
> > saying
> > > "rebalance
> > > timeout shall not remove current members, only session timeout will
> do."
> > > As you could see this is not an ideal scenario: we trigger extra
> > rebalance
> > > at step 5. In my reply to Guozhang I'm asking whether we should still
> use
> > > LeaveGroupRequest for static members to send a signal to broker saying
> > "I'm
> > > currently offline", and when we send ConsumerRebalanceRequest to
> broker,
> > we
> > > will actually kick off c4 because it says it's offline already, saving
> > one
> > > or multiple additional rebalances later. This way the
> > > ConsumerRebalanceRequest will be more effective in making correct
> > judgement
> > > on the group status since we have more feedback from client side.
> > >
> > > > - When we say that we would use invokeConsumerRebalance(groupId) to
> > down
> > > > scale, with the example in the above question, how will the
> > > > GroupCoordinator know that c4 should be kicked out of the group since
> > we
> > > > are trying to invoke rebalance proactively without waiting for c4's
> > > session
> > > > time out to expire. Should there be a way of telling the
> > GroupCoordinator
> > > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > > Previous proposal should be suffice to answer this question 😊
> > >
> > > - Also it looks like the statement "If the `member.id` uses
> > > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> > replace
> > > > the one within current map, if `group.member.name` is known. Also
> once
> > > we
> > > > are done with KIP-394
> > > > <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > > > >,
> > > > all the join group requests are requiring `member.id` to physically
> > > enter
> > > > the consumer group. This way the latest joined " is incomplete. Can
> you
> > > > take a look at this?
> > > > Also when we say "all the join group requests are requiring `
> member.id
> > `
> > > to
> > > > physically enter the consumer group." because a newly started
> consumer
> > > will
> > > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > > assigns a member.id to the newly started consumer, it has to use it
> > for
> > > > any
> > > > future JoinGroupRequests. Is my understanding correct?
> > > >
> > > Thanks for catching it! And yes, we shall use one extra round-trip
> > between
> > > consumer
> > > and broker to inform the new member id allocation.
> > >
> > > Next is the replies to Guozhang's comment:
> > > 2) I once have a discussion about the LeaveGroupRequest for static
> > members,
> > > and the reason for not having it for static members is that we'd need
> to
> > > make it a configurable behavior as well (i.e. the likelihood that a
> > static
> > > member may shutdown but come back later may be even larger than the
> > > likelihood that a shutdown static member would not come back), and
> when a
> > > shutdown is complete the instance cannot tell whether or not it will
> come
> > > back by itself. And hence letting a third party (think: admin used by
> K8s
> > > plugins) issuing a request to indicate static member changes would be
> > more
> > > plausible.
> > >
> > > I think having an optional list of all the static members that are
> still
> > in
> > > the group, rather than the members to be removed since the latter
> looks a
> > > bit less flexible to me, in the request is a good idea (remember we
> > allow a
> > > group to have both static and dynamic members at the same time, so when
> > > receiving the request, we will only do the diff and add / remove the
> > static
> > > members directly only, while still let the dynamic members to try to
> > > re-join the group with the rebalance timeout).
> > > I'm also in favor of storing all the in-group static members. In fact
> we
> > > could reuse
> > > the static membership mapping to store this information. Do you think
> > > that we should let static member send leave group request to indicate
> > > their status of "leaving",
> > > and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> > > suggesting we should
> > > remove those members when kicking off rebalance since we are shutting
> > them
> > > down already.
> > >
> > > 3) personally I favor "ids" over "names" :) Since we already have some
> > > "ids" and hence it sounds more consistent, plus on the producer side we
> > > have a `transactional.id` whose semantics is a bit similar to this
> one,
> > > i.e. for unique distinguishment of a client which may comes and goes
> but
> > > need to be persist over multiple "instance life-times".
> > > Sure we have enough votes for ids 😊I will finalize the name to `
> > > group.instance.id`, does that
> > > sound good?
> > >
> > > Best,
> > > Boyang
> > > ________________________________
> > > From: Guozhang Wang <wa...@gmail.com>
> > > Sent: Wednesday, November 28, 2018 4:51 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Regarding Jason's question and Boyang's responses:
> > >
> > > 2) I once have a discussion about the LeaveGroupRequest for static
> > members,
> > > and the reason for not having it for static members is that we'd need
> to
> > > make it a configurable behavior as well (i.e. the likelihood that a
> > static
> > > member may shutdown but come back later may be even larger than the
> > > likelihood that a shutdown static member would not come back), and
> when a
> > > shutdown is complete the instance cannot tell whether or not it will
> come
> > > back by itself. And hence letting a third party (think: admin used by
> K8s
> > > plugins) issuing a request to indicate static member changes would be
> > more
> > > plausible.
> > >
> > > I think having an optional list of all the static members that are
> still
> > in
> > > the group, rather than the members to be removed since the latter
> looks a
> > > bit less flexible to me, in the request is a good idea (remember we
> > allow a
> > > group to have both static and dynamic members at the same time, so when
> > > receiving the request, we will only do the diff and add / remove the
> > static
> > > members directly only, while still let the dynamic members to try to
> > > re-join the group with the rebalance timeout).
> > >
> > > 3) personally I favor "ids" over "names" :) Since we already have some
> > > "ids" and hence it sounds more consistent, plus on the producer side we
> > > have a `transactional.id` whose semantics is a bit similar to this
> one,
> > > i.e. for unique distinguishment of a client which may comes and goes
> but
> > > need to be persist over multiple "instance life-times".
> > >
> > >
> > > Guozhang
> > >
> > >
> > > On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the replies. Please find the follow up queries below.
> > > >
> > > >     5. Regarding "So in summary, *the member will only be removed due
> > to
> > > > session timeout*. We shall remove it from both in-memory static
> member
> > > name
> > > > mapping and member list." If the rebalance is invoked manually using
> > the
> > > > the admin apis, how long should the group coordinator wait for the
> > > members
> > > > of the group to send a JoinGroupRequest for participating in the
> > > rebalance?
> > > > How is a lagging consumer handled?
> > > > The plan is to disable member kick out when rebalance.timeout is
> > reached,
> > > > so basically we are not "waiting" any
> > > > join group request from existing members; we shall just rebalance
> base
> > on
> > > > what we currently have within the group
> > > > metadata. Lagging consumer will trigger rebalance later if session
> > > timeout
> > > > > rebalance timeout.
> > > >
> > > > >
> > > > Just wanted to understand this better. Lets take an example, say we
> > have
> > > a
> > > > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > > > Everything is running fine and suddenly C4 host has issues and it
> > goes
> > > > > down. Now we notice that we can still operate with c1, c2, c3 and
> > don't
> > > > > want to wait for
> > > > > c4 to come back up. We use the admin api
> > > > > "invokeConsumerRebalance("GroupA")".
> > > > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join
> the
> > > > > group again (in there heartBeatResponse) as first step of
> rebalance.
> > > > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3
> > is
> > > > > delayed. At this stage, if we are not "waiting" on any join group
> > > > request,
> > > > > few things can happen :
> > > > >
> > > > >    - c4's partitions are distributed only among c1,c2. c3 maintains
> > its
> > > > >    original assignment. c1, c2 will start processing the newly
> > assigned
> > > > >    partitions.
> > > > >
> > > > > OR
> > > > >
> > > > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2
> > start
> > > > >    processing the newly assigned partitions. c3 gets to know about
> > the
> > > > newly
> > > > >    assigned partitions later when it sends the JoinGroupRequest
> > (which
> > > > was
> > > > >    delayed).
> > > > >
> > > > > OR
> > > > >
> > > > >    - Will the rebalance do a complete reassignment, where c1, c2,
> c3
> > > have
> > > > >    to give up there partitions and all the partitions belonging to
> > c1,
> > > > c2, c3,
> > > > >    c4 will be redistributed among c1, c2, c3 ? If this is the case,
> > the
> > > > >    GroupCoordinator needs to give some buffer time for c1, c2, c3
> to
> > > > revoke
> > > > >    there partitions and rejoin the group.
> > > > >
> > > > > This is as per my understanding of how the KIP would work without
> > > > changing
> > > > > the underlying group coordination workflow. Please correct me if I
> > > > > misunderstood something here.
> > > > >
> > > >
> > > >
> > > > - When we say that we would use invokeConsumerRebalance(groupId) to
> > down
> > > > scale, with the example in the above question, how will the
> > > > GroupCoordinator know that c4 should be kicked out of the group since
> > we
> > > > are trying to invoke rebalance proactively without waiting for c4's
> > > session
> > > > time out to expire. Should there be a way of telling the
> > GroupCoordinator
> > > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > > >
> > > > - Also it looks like the statement "If the `member.id` uses
> > > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> > replace
> > > > the one within current map, if `group.member.name` is known. Also
> once
> > > we
> > > > are done with KIP-394
> > > > <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > > > >,
> > > > all the join group requests are requiring `member.id` to physically
> > > enter
> > > > the consumer group. This way the latest joined " is incomplete. Can
> you
> > > > take a look at this?
> > > > Also when we say "all the join group requests are requiring `
> member.id
> > `
> > > to
> > > > physically enter the consumer group." because a newly started
> consumer
> > > will
> > > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > > assigns a member.id to the newly started consumer, it has to use it
> > for
> > > > any
> > > > future JoinGroupRequests. Is my understanding correct?
> > > >
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> > > both
> > > > > in this reply.
> > > > >
> > > > >
> > > > > >    1. Do you intend to have member.id is a static config like
> > > > > member.name
> > > > > >    after KIP-345 and KIP-394?
> > > > >
> > > > > No, we shall only rely on broker to allocate member.id for the
> > > consumer
> > > > > instances. FYI, I already
> > > > >
> > > > > started the discussion thread for KIP-394 😊
> > > > >
> > > > > >    2. Regarding "On client side, we add a new config called
> > > MEMBER_NAME
> > > > > in
> > > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > > config
> > > > is
> > > > > > set,
> > > > > >    we will put it in the initial join group request to identify
> > > itself
> > > > > as a
> > > > > >    static member (static membership); otherwise, we will still
> send
> > > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > > (dynamic
> > > > > >    membership)."
> > > > > >       - What is the value of member_id sent in the first
> > > > JoinGroupRequest
> > > > > >       when member_name is set (using static rebalance)? Is it
> > > > > > UNKNOW_MEMBER_ID?
> > > > >
> > > > > Yes, we could only use unknown member id. Actually this part of the
> > > > > proposal is outdated,
> > > > >
> > > > > let me do another audit of the whole doc. Basically, it is
> currently
> > > > > impossible to send `member.id`
> > > > >
> > > > > when consumer restarted. Sorry for the confusions!
> > > > >
> > > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > > match
> > > > > the
> > > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > > edge
> > > > > case
> > > > > >    that if we could have members with the same `member.name`
> (for
> > > > > example
> > > > > >    mis-configured instances with a valid member.id but added a
> > used
> > > > > member
> > > > > >    name on runtime). When member name has duplicates, we could
> > refuse
> > > > > join
> > > > > >    request from members with an outdated `member.id` (since we
> > > update
> > > > > the
> > > > > >    mapping upon each join group request). In an edge case where
> the
> > > > > client
> > > > > >    hits this exception in the response, it is suggesting that
> some
> > > > other
> > > > > >    consumer takes its spot."
> > > > > >       - The part of "some other consumer takes the spot" would be
> > > > > >       intentional, right? Also when you say " The edge case that
> if
> > > we
> > > > > >       could have members with the same `member.name` (for
> example
> > > > > >       mis-configured instances *with a valid member.id <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > > >
> > > > > > *but
> > > > > >       added a used member name on runtime).", what do you mean by
> > > > *valid
> > > > > >       member id* here? Does it mean that there exist a mapping of
> > > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > > member.name
> > > > > >       <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > > >
> > > > > = MemberB and member.id <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > >
> > > > > =
> > > > > > id1 *
> > > > > >       ?
> > > > >
> > > > > I would take Jason's advice that each time we have unknown member
> > > joining
> > > > > the group, the broker will
> > > > >
> > > > > always assign a new and unique id to track its identity. In this
> way,
> > > > > consumer with duplicate member name
> > > > >
> > > > > will be fenced.
> > > > >
> > > > > >    4. Depending on your explanation for point 2 and the point 3
> > above
> > > > > >    regarding returning back MEMBER_ID_MISMATCH on having a
> matching
> > > > > >    member_name but unknown member_id, if the consumer sends
> > > > > > "UNKNOW_MEMBER_ID"
> > > > > >    on the first JoinGroupRequest and relies on the
> GroupCoordinator
> > > to
> > > > > > give it
> > > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > > >
> > > > > Like explained above, we shall not materialize the member.id.
> > Instead
> > > we
> > > > > need to rely on broker to allocate
> > > > >
> > > > > a unique id for consumer just like what we have now.
> > > > >
> > > > > >    5. Regarding "So in summary, *the member will only be removed
> > due
> > > to
> > > > > >    session timeout*. We shall remove it from both in-memory
> static
> > > > member
> > > > > >    name mapping and member list."
> > > > > >       - If the rebalance is invoked manually using the the admin
> > > apis,
> > > > > how
> > > > > >       long should the group coordinator wait for the members of
> the
> > > > > > group to send
> > > > > >       a JoinGroupRequest for participating in the rebalance? How
> > is a
> > > > > > lagging
> > > > > >       consumer handled?
> > > > >
> > > > > The plan is to disable member kick out when rebalance.timeout is
> > > reached,
> > > > > so basically we are not "waiting" any
> > > > >
> > > > > join group request from existing members; we shall just rebalance
> > base
> > > on
> > > > > what we currently have within the group
> > > > >
> > > > > metadata. Lagging consumer will trigger rebalance later if session
> > > > timeout
> > > > > > rebalance timeout.
> > > > >
> > > > > >    6. Another detail to take care is that we need to
> automatically
> > > take
> > > > > the
> > > > > >    hash of group id so that we know which broker to send this
> > request
> > > > to.
> > > > > >       - I assume this should be same as the way we find the
> > > > coordinator,
> > > > > >       today right? If yes, should we specify it in the KIP ?
> > > > >
> > > > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > > > >
> > > > > >    7. Are there any specific failure scenarios when you say
> "other
> > > > > >    potential failure cases."? It would be good to mention them
> > > > > explicitly,
> > > > > > if
> > > > > >    you think there are any.
> > > > >
> > > > > Nah, I'm gonna remove it because it seems causing more confusion
> than
> > > > > making my assumption clear, which is
> > > > >
> > > > > "there could be other failure cases that I can't enumerate now" 😊
> > > > >
> > > > > >    8. It would be good to have a rollback plan as you have for
> roll
> > > > > forward
> > > > > >    in the KIP.
> > > > >
> > > > > Great suggestion! Added a simple rollback plan.
> > > > >
> > > > >
> > > > > Next is answering Jason's suggestions:
> > > > >
> > > > > 1. This may be the same thing that Mayuresh is asking about. I
> think
> > > the
> > > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > > member
> > > > > name, but no member id, then we will return the current member id
> > > > > associated with that name. It seems in this case that we wouldn't
> be
> > > able
> > > > > to protect from having two consumers active with the same
> configured
> > > > > member.name? For example, imagine that we had a consumer with
> > > > member.name
> > > > > =A
> > > > > which is assigned member.id=1. Suppose it becomes a zombie and a
> new
> > > > > instance starts up with member.name=A. If it is also assigned
> > > member.id
> > > > =1,
> > > > > then how can we detect the zombie if it comes back to life? Both
> > > > instances
> > > > > will have the same member.id.
> > > > >
> > > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > > need
> > > > to
> > > > > fence previous members. I am wondering if we can generate a new
> > > > member.id
> > > > > every time we receive a request from a static member with an
> unknown
> > > > member
> > > > > id. If the old instance with the same member.name attempts any
> > > > operation,
> > > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> > the
> > > > > subscription of the new instance hasn't changed, then we can skip
> the
> > > > > rebalance and return the current assignment without forcing a
> > > rebalance.
> > > > >
> > > > > The trick to making this work is in the error handling of the
> zombie
> > > > > consumer. If the zombie simply resets its member.id and rejoins to
> > > get a
> > > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would
> end
> > > up
> > > > > fencing the new member. We want to avoid this. There needs to be an
> > > > > expectation for static members that the member.id of a static
> member
> > > > will
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the
> > > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > > consumers
> > > > > with static member names.
> > > > >
> > > > > Yep, I like this idea! Keep giving out refresh member.id when
> facing
> > > > > anonymous request will definitely
> > > > >
> > > > > prevent processing bug due to duplicate consumers, however I don't
> > > think
> > > > I
> > > > > fully understand the 3rd paragraph where
> > > > >
> > > > > you mentioned  "There needs to be an expectation for static members
> > > that
> > > > > the member.id of a static member will
> > > > >
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the group. "  How do you plan
> > > > > to know whether this member is new member or old member? I feel
> even
> > > with
> > > > > zombie consumer takes the ownership,
> > > > > it should be detected very quickly (as MISMATCH_ID exception
> trigger
> > > > > original consumer instance dies)
> > > > > and end user will start to fix it right away. Is there any similar
> > > logic
> > > > > we applied in fencing duplicate `transaction.id`?
> > > > >
> > > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me.
> As
> > > far
> > > > as
> > > > > I understand it, it is used for scaling down a consumer group and
> > > somehow
> > > > > bypasses normal session timeout expiration. I am wondering how
> > critical
> > > > > this piece is and whether we can leave it for future work. If not,
> > then
> > > > it
> > > > > would be helpful to elaborate on its implementation. How would the
> > > > > coordinator know which members to kick out of the group?
> > > > >
> > > > > This API is needed when we need to immediately trigger rebalance
> > > instead
> > > > > of waiting session timeout
> > > > >
> > > > > or rebalance timeout (Emergent scale up/down). It is very necessary
> > to
> > > > > have it for
> > > > >
> > > > > management purpose because user could choose when to trigger
> > rebalance
> > > > > pretty freely,
> > > > >
> > > > > gaining more client side control.
> > > > >
> > > > > In the meanwhile I see your point that we need to actually have the
> > > > > ability to kick out members that we plan
> > > > >
> > > > > to scale down fast (as rebalance timeout no longer kicks any
> offline
> > > > > member out of the group), I will think of adding an optional
> > > > >
> > > > > list of members that are ready to be removed.
> > > > >
> > > > > Another idea is to let static member send `LeaveGroupRequest` when
> > they
> > > > > are going offline (either scale down or bouncing),
> > > > >
> > > > > and broker will cache this information as "OfflineMembers" without
> > > > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> > > will
> > > > >
> > > > > kick the static members that are currently offline and trigger
> > > rebalance
> > > > > immediately. How does this plan sound?
> > > > >
> > > > > 3. I've been holding back on mentioning this, but I think we should
> > > > > reconsider the name `member.name`. I think we want something that
> > > > suggests
> > > > > its expectation of uniqueness in the group. How about `
> > > group.instance.id
> > > > `
> > > > > to go along with `group.id`?
> > > > >
> > > > > Yea, Dong and Stanislav also mentioned this naming. I personally
> buy
> > in
> > > > > the namespace idea, and
> > > > >
> > > > > since we already use `member.name` in a lot of context, I decide
> to
> > > > > rename the config to `group.member.name`
> > > > >
> > > > > which should be sufficient for solving all the concerns we have
> now.
> > > > > Sounds good?
> > > > >
> > > > >
> > > > > Thank you for your great suggestions! Let me know if my reply makes
> > > sense
> > > > > her.
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Jason Gustafson <ja...@confluent.io>
> > > > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > > > To: dev
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the updates. Looks like we're headed in the right
> > direction
> > > > and
> > > > > clearly the interest that this KIP is receiving shows how strong
> the
> > > > > motivation is!
> > > > >
> > > > > I have a few questions:
> > > > >
> > > > > 1. This may be the same thing that Mayuresh is asking about. I
> think
> > > the
> > > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > > member
> > > > > name, but no member id, then we will return the current member id
> > > > > associated with that name. It seems in this case that we wouldn't
> be
> > > able
> > > > > to protect from having two consumers active with the same
> configured
> > > > > member.name? For example, imagine that we had a consumer with
> > > > member.name
> > > > > =A
> > > > > which is assigned member.id=1. Suppose it becomes a zombie and a
> new
> > > > > instance starts up with member.name=A. If it is also assigned
> > > member.id
> > > > =1,
> > > > > then how can we detect the zombie if it comes back to life? Both
> > > > instances
> > > > > will have the same member.id.
> > > > >
> > > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > > need
> > > > to
> > > > > fence previous members. I am wondering if we can generate a new
> > > > member.id
> > > > > every time we receive a request from a static member with an
> unknown
> > > > member
> > > > > id. If the old instance with the same member.name attempts any
> > > > operation,
> > > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> > the
> > > > > subscription of the new instance hasn't changed, then we can skip
> the
> > > > > rebalance and return the current assignment without forcing a
> > > rebalance.
> > > > >
> > > > > The trick to making this work is in the error handling of the
> zombie
> > > > > consumer. If the zombie simply resets its member.id and rejoins to
> > > get a
> > > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would
> end
> > > up
> > > > > fencing the new member. We want to avoid this. There needs to be an
> > > > > expectation for static members that the member.id of a static
> member
> > > > will
> > > > > not be changed except when a new member with the same member.name
> > > joins
> > > > > the
> > > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > > consumers
> > > > > with static member names.
> > > > >
> > > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me.
> As
> > > far
> > > > as
> > > > > I understand it, it is used for scaling down a consumer group and
> > > somehow
> > > > > bypasses normal session timeout expiration. I am wondering how
> > critical
> > > > > this piece is and whether we can leave it for future work. If not,
> > then
> > > > it
> > > > > would be helpful to elaborate on its implementation. How would the
> > > > > coordinator know which members to kick out of the group?
> > > > >
> > > > > 3. I've been holding back on mentioning this, but I think we should
> > > > > reconsider the name `member.name`. I think we want something that
> > > > suggests
> > > > > its expectation of uniqueness in the group. How about `
> > > group.instance.id
> > > > `
> > > > > to go along with `group.id`?
> > > > >
> > > > > Thanks,
> > > > > Jason
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > > > gharatmayuresh15@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > Thanks a lot for replying to all the queries and discussions
> here,
> > so
> > > > > > patiently.
> > > > > > Really appreciate it.
> > > > > >
> > > > > > Had a few questions and suggestions after rereading the current
> > > version
> > > > > of
> > > > > > the KIP :
> > > > > >
> > > > > >
> > > > > >    1. Do you intend to have member.id is a static config like
> > > > > member.name
> > > > > >    after KIP-345 and KIP-394?
> > > > > >    2. Regarding "On client side, we add a new config called
> > > MEMBER_NAME
> > > > > in
> > > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > > config
> > > > is
> > > > > > set,
> > > > > >    we will put it in the initial join group request to identify
> > > itself
> > > > > as a
> > > > > >    static member (static membership); otherwise, we will still
> send
> > > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > > (dynamic
> > > > > >    membership)."
> > > > > >       - What is the value of member_id sent in the first
> > > > JoinGroupRequest
> > > > > >       when member_name is set (using static rebalance)? Is it
> > > > > > UNKNOW_MEMBER_ID?
> > > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > > match
> > > > > the
> > > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > > edge
> > > > > case
> > > > > >    that if we could have members with the same `member.name`
> (for
> > > > > example
> > > > > >    mis-configured instances with a valid member.id but added a
> > used
> > > > > member
> > > > > >    name on runtime). When member name has duplicates, we could
> > refuse
> > > > > join
> > > > > >    request from members with an outdated `member.id` (since we
> > > update
> > > > > the
> > > > > >    mapping upon each join group request). In an edge case where
> the
> > > > > client
> > > > > >    hits this exception in the response, it is suggesting that
> some
> > > > other
> > > > > >    consumer takes its spot."
> > > > > >       - The part of "some other consumer takes the spot" would be
> > > > > >       intentional, right? Also when you say " The edge case that
> if
> > > we
> > > > > >       could have members with the same `member.name` (for
> example
> > > > > >       mis-configured instances *with a valid member.id <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > > >
> > > > > > *but
> > > > > >       added a used member name on runtime).", what do you mean by
> > > > *valid
> > > > > >       member id* here? Does it mean that there exist a mapping of
> > > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > > member.name
> > > > > >       <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > > >
> > > > > = MemberB and member.id <
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > >
> > > > > =
> > > > > > id1 *
> > > > > >       ?
> > > > > >    4. Depending on your explanation for point 2 and the point 3
> > above
> > > > > >    regarding returning back MEMBER_ID_MISMATCH on having a
> matching
> > > > > >    member_name but unknown member_id, if the consumer sends
> > > > > > "UNKNOW_MEMBER_ID"
> > > > > >    on the first JoinGroupRequest and relies on the
> GroupCoordinator
> > > to
> > > > > > give it
> > > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > > > >    5. Regarding "So in summary, *the member will only be removed
> > due
> > > to
> > > > > >    session timeout*. We shall remove it from both in-memory
> static
> > > > member
> > > > > >    name mapping and member list."
> > > > > >       - If the rebalance is invoked manually using the the admin
> > > apis,
> > > > > how
> > > > > >       long should the group coordinator wait for the members of
> the
> > > > > > group to send
> > > > > >       a JoinGroupRequest for participating in the rebalance? How
> > is a
> > > > > > lagging
> > > > > >       consumer handled?
> > > > > >    6. Another detail to take care is that we need to
> automatically
> > > take
> > > > > the
> > > > > >    hash of group id so that we know which broker to send this
> > request
> > > > to.
> > > > > >       - I assume this should be same as the way we find the
> > > > coordinator,
> > > > > >       today right? If yes, should we specify it in the KIP ?
> > > > > >    7. Are there any specific failure scenarios when you say
> "other
> > > > > >    potential failure cases."? It would be good to mention them
> > > > > explicitly,
> > > > > > if
> > > > > >    you think there are any.
> > > > > >    8. It would be good to have a rollback plan as you have for
> roll
> > > > > forward
> > > > > >    in the KIP.
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Mayuresh
> > > > > >
> > > > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > > > gharatmayuresh15@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi Boyang,
> > > > > > >
> > > > > > > Do you have a discuss thread for KIP-394 that you mentioned
> here
> > ?
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Mayuresh
> > > > > > >
> > > > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <
> bchen11@outlook.com
> > >
> > > > > wrote:
> > > > > > >
> > > > > > >> Hey Dong, thanks for the follow-up here!
> > > > > > >>
> > > > > > >>
> > > > > > >> 1) It is not very clear to the user what is the difference
> > between
> > > > > > >> member.name and client.id as both seems to be used to
> identify
> > > the
> > > > > > >> consumer. I am wondering if it would be more intuitive to name
> > it
> > > > > > >> group.member.name (preferred choice since it matches the
> > current
> > > > > > group.id
> > > > > > >> config name) or rebalance.member.name to explicitly show that
> > the
> > > > id
> > > > > is
> > > > > > >> solely used for rebalance.
> > > > > > >> Great question. I feel `member.name` is enough to explain
> > itself,
> > > > it
> > > > > > >> seems not very
> > > > > > >> helpful to make the config name longer. Comparing `name` with
> > `id`
> > > > > gives
> > > > > > >> user the
> > > > > > >> impression that they have the control over it with customized
> > rule
> > > > > than
> > > > > > >> library decided.
> > > > > > >>
> > > > > > >> 2) In the interface change section it is said that
> > > > > > >> GroupMaxSessionTimeoutMs
> > > > > > >> will be changed to 30 minutes. It seems to suggest that we
> will
> > > > change
> > > > > > the
> > > > > > >> default value of this config. It does not seem necessary to
> > > increase
> > > > > the
> > > > > > >> time of consumer failure detection when user doesn't use
> static
> > > > > > >> membership.
> > > > > > >> Also, say static membership is enabled, then this default
> config
> > > > > change
> > > > > > >> will cause a partition to be unavailable for consumption for
> 30
> > > > > minutes
> > > > > > if
> > > > > > >> there is hard consumer failure, which seems to be worse
> > experience
> > > > > than
> > > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > > particularly
> > > > > > >> for
> > > > > > >> new users of Kafka. Could you explain more why we should make
> > this
> > > > > > change?
> > > > > > >> We are not changing the default session timeout value. We are
> > just
> > > > > > >> changing the
> > > > > > >> cap we are enforcing on the session timeout max value. So this
> > > > change
> > > > > is
> > > > > > >> not affecting
> > > > > > >> what kind of membership end user is using, and loosing the cap
> > is
> > > > > giving
> > > > > > >> end user
> > > > > > >> more flexibility on trade-off between liveness and stability.
> > > > > > >>
> > > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > > DUPLICATE_STATIC_MEMBER
> > > > > > >> into one error? It seems that these two errors are currently
> > > handled
> > > > > by
> > > > > > >> the
> > > > > > >> consumer in the same way. And we don't also don't expect
> > > > > > >> MEMBER_ID_MISMATCH
> > > > > > >> to happen. Thus it is not clear what is the benefit of having
> > two
> > > > > > errors.
> > > > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> > > because
> > > > > with
> > > > > > >> the KIP-394<
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=E3cqYTMRFsAs5TQI4JxHm3kOWCfkVWjpuc%2BuNHezwG0%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> we will automatically fence all join requests with
> > > > UNKNOWN_MEMBER_ID.
> > > > > > >>
> > > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join
> group
> > > > > > contains
> > > > > > >> member name which is already in the consumer group, however
> the
> > > > member
> > > > > > id
> > > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > > >> JoinGroupRequest with an existing memberName (as the
> coordinator
> > > has
> > > > > not
> > > > > > >> expired this member from the memory) and memberId
> > > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > > persisted
> > > > > > >> across consumer restart in the consumer side). Does it mean
> that
> > > > > > >> JoinGroupRequest from a newly restarted consumer will always
> be
> > > > > rejected
> > > > > > >> until the sessionTimeoutMs has passed?
> > > > > > >> Same answer as question 3). This part of the logic shall be
> > > removed
> > > > > from
> > > > > > >> the proposal.
> > > > > > >>
> > > > > > >> 5) It seems that we always add two methods to the interface
> > > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> > options
> > > > and
> > > > > > the
> > > > > > >> other without option. Could this be specified in the interface
> > > > change
> > > > > > >> section?
> > > > > > >> Sounds good! Added both methods.
> > > > > > >>
> > > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE
> to
> > > > > trigger
> > > > > > >> rebalance? If so, we probably want to specify the command line
> > > tool
> > > > > > >> interface similar to
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > > > >> .
> > > > > > >> Added the script.
> > > > > > >>
> > > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> > with
> > > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> > extra
> > > > > > meaning
> > > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > > seems
> > > > > > >> simpler
> > > > > > >> to allows this API to trigger rebalance regardless of whether
> > > > consumer
> > > > > > is
> > > > > > >> configured with memberName.
> > > > > > >> Sounds good. Right now I feel for both static and dynamic
> > > membership
> > > > > it
> > > > > > is
> > > > > > >> more manageable to introduce the consumer rebalance method
> > through
> > > > > admin
> > > > > > >> client API.
> > > > > > >>
> > > > > > >> 8) It is not very clear how the newly added AdminClient API
> > > trigger
> > > > > > >> rebalance. For example, does it send request? Can this be
> > > explained
> > > > in
> > > > > > the
> > > > > > >> KIP?
> > > > > > >>
> > > > > > >> Sure, I will add more details to the API.
> > > > > > >>
> > > > > > >>
> > > > > > >> Thanks again for the helpful suggestions!
> > > > > > >>
> > > > > > >>
> > > > > > >> Best,
> > > > > > >> Boyang
> > > > > > >>
> > > > > > >> ________________________________
> > > > > > >> From: Dong Lin <li...@gmail.com>
> > > > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > > > >> To: dev
> > > > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > by
> > > > > > >> specifying member id
> > > > > > >>
> > > > > > >> Hey Boyang,
> > > > > > >>
> > > > > > >> Thanks for the update! Here are some followup comments:
> > > > > > >>
> > > > > > >> 1) It is not very clear to the user what is the difference
> > between
> > > > > > >> member.name and client.id as both seems to be used to
> identify
> > > the
> > > > > > >> consumer. I am wondering if it would be more intuitive to name
> > it
> > > > > > >> group.member.name (preferred choice since it matches the
> > current
> > > > > > group.id
> > > > > > >> config name) or rebalance.member.name to explicitly show that
> > the
> > > > id
> > > > > is
> > > > > > >> solely used for rebalance.
> > > > > > >>
> > > > > > >> 2) In the interface change section it is said that
> > > > > > >> GroupMaxSessionTimeoutMs
> > > > > > >> will be changed to 30 minutes. It seems to suggest that we
> will
> > > > change
> > > > > > the
> > > > > > >> default value of this config. It does not seem necessary to
> > > increase
> > > > > the
> > > > > > >> time of consumer failure detection when user doesn't use
> static
> > > > > > >> membership.
> > > > > > >> Also, say static membership is enabled, then this default
> config
> > > > > change
> > > > > > >> will cause a partition to be unavailable for consumption for
> 30
> > > > > minutes
> > > > > > if
> > > > > > >> there is hard consumer failure, which seems to be worse
> > experience
> > > > > than
> > > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > > particularly
> > > > > > >> for
> > > > > > >> new users of Kafka. Could you explain more why we should make
> > this
> > > > > > change?
> > > > > > >>
> > > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > > DUPLICATE_STATIC_MEMBER
> > > > > > >> into one error? It seems that these two errors are currently
> > > handled
> > > > > by
> > > > > > >> the
> > > > > > >> consumer in the same way. And we don't also don't expect
> > > > > > >> MEMBER_ID_MISMATCH
> > > > > > >> to happen. Thus it is not clear what is the benefit of having
> > two
> > > > > > errors.
> > > > > > >>
> > > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join
> group
> > > > > > contains
> > > > > > >> member name which is already in the consumer group, however
> the
> > > > member
> > > > > > id
> > > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > > >> JoinGroupRequest with an existing memberName (as the
> coordinator
> > > has
> > > > > not
> > > > > > >> expired this member from the memory) and memberId
> > > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > > persisted
> > > > > > >> across consumer restart in the consumer side). Does it mean
> that
> > > > > > >> JoinGroupRequest from a newly restarted consumer will always
> be
> > > > > rejected
> > > > > > >> until the sessionTimeoutMs has passed?
> > > > > > >>
> > > > > > >> 5) It seems that we always add two methods to the interface
> > > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> > options
> > > > and
> > > > > > the
> > > > > > >> other without option. Could this be specified in the interface
> > > > change
> > > > > > >> section?
> > > > > > >>
> > > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE
> to
> > > > > trigger
> > > > > > >> rebalance? If so, we probably want to specify the command line
> > > tool
> > > > > > >> interface similar to
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > > > >> .
> > > > > > >>
> > > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> > with
> > > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> > extra
> > > > > > meaning
> > > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > > seems
> > > > > > >> simpler
> > > > > > >> to allows this API to trigger rebalance regardless of whether
> > > > consumer
> > > > > > is
> > > > > > >> configured with memberName.
> > > > > > >>
> > > > > > >> 8) It is not very clear how the newly added AdminClient API
> > > trigger
> > > > > > >> rebalance. For example, does it send request? Can this be
> > > explained
> > > > in
> > > > > > the
> > > > > > >> KIP?
> > > > > > >>
> > > > > > >> Thanks,
> > > > > > >> Dong
> > > > > > >>
> > > > > > >>
> > > > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <
> > bchen11@outlook.com>
> > > > > > wrote:
> > > > > > >>
> > > > > > >> > Hey Mayuresh,
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > thanks for your feedbacks! I will try do another checklist
> > here.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > By this you mean, even if the application has not called
> > > > > > >> > > KafkaConsumer.poll() within session timeout, it will not
> be
> > > > > sending
> > > > > > >> the
> > > > > > >> > > LeaveGroup request, right?
> > > > > > >> >
> > > > > > >> > Yep it's true, we will prevent client from sending leave
> group
> > > > > request
> > > > > > >> > when they are set with `member.name`.
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > > When is the member.name removed from this map?
> > > > > > >> > Good question, we will only kick off member due to session
> > > timeout
> > > > > > >> within
> > > > > > >> > static membership. Let me update the KIP to clearly assert
> > that.
> > > > > > >> >
> > > > > > >> > > How is this case (missing member id) handled on the client
> > > side?
> > > > > > What
> > > > > > >> is
> > > > > > >> > the application that
> > > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > > >> > I have extended the two exceptions within join group
> response
> > > V4.
> > > > > > >> > Basically I define both corresponding actions to be
> immediate
> > > > > failing
> > > > > > >> > client application, because so far it is unknown what kind
> of
> > > > client
> > > > > > >> issue
> > > > > > >> > could trigger them. After the first version, we will keep
> > > enhance
> > > > > the
> > > > > > >> error
> > > > > > >> > handling logic!
> > > > > > >> >
> > > > > > >> > > This would mean that it might take more time to detect
> > unowned
> > > > > topic
> > > > > > >> > > partitions and may cause delay for applications that
> perform
> > > > data
> > > > > > >> > mirroring
> > > > > > >> > > tasks. I discussed this with our sre and we have a
> > suggestion
> > > to
> > > > > > make
> > > > > > >> > here
> > > > > > >> > > as listed below separately.
> > > > > > >> > The goal of extending session timeout cap is for users with
> > good
> > > > > > client
> > > > > > >> > side monitoring tools that could auto-heal the dead
> consumers
> > > very
> > > > > > >> fast. So
> > > > > > >> > it is optional (and personal) to extend session timeout to a
> > > > > > reasonable
> > > > > > >> > number with different client scenarios.
> > > > > > >> >
> > > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > > > >> >
> > > > > > >> > > What do you mean by " Internally we would optimize this
> > logic
> > > by
> > > > > > >> having
> > > > > > >> > > rebalance timeout only in charge of stopping prepare
> > rebalance
> > > > > > stage,
> > > > > > >> > > without removing non-responsive members immediately."
> There
> > > > would
> > > > > > not
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > > request
> > > > > > later,
> > > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > > > functionality
> > > > > > >> > to only use as a timer to
> > > > > > >> > end prepare rebalance stage. This way, late joining static
> > > members
> > > > > > will
> > > > > > >> > not trigger further rebalance
> > > > > > >> > as long as they are within session timeout. I added your
> > > highlight
> > > > > to
> > > > > > >> the
> > > > > > >> > KIP!
> > > > > > >> >
> > > > > > >> > > The KIP talks about scale up scenario but its not quite
> > clear
> > > > how
> > > > > we
> > > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> > we
> > > > > adding
> > > > > > >> > status
> > > > > > >> > > "learner" ?. Can you shed more light on how this is
> handled
> > in
> > > > the
> > > > > > >> KIP,
> > > > > > >> > if
> > > > > > >> > > its handled?
> > > > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> > > because
> > > > we
> > > > > > >> > believe client side could
> > > > > > >> > better handle this logic.
> > > > > > >> >
> > > > > > >> > > I think Jason had brought this up earlier about having a
> way
> > > to
> > > > > say
> > > > > > >> how
> > > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > > consumer
> > > > > > >> group.
> > > > > > >> > > If we can do this, then in case of mirroring applications
> we
> > > can
> > > > > do
> > > > > > >> this
> > > > > > >> > :
> > > > > > >> > > Lets say we have a mirroring application that consumes
> from
> > > > Kafka
> > > > > > >> cluster
> > > > > > >> > > A and produces to Kafka cluster B.
> > > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > > Kafka
> > > > > > >> service
> > > > > > >> > > providers can set a mirroring group saying that it will
> > take,
> > > > for
> > > > > > >> example
> > > > > > >> > > 300 consumer hosts/members to achieve the desired
> throughput
> > > and
> > > > > > >> latency
> > > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > > spare
> > > > > in
> > > > > > >> the
> > > > > > >> > > same group.
> > > > > > >> > > So when the first 300 members/consumers to join the group
> > will
> > > > > start
> > > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster
> B.
> > > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > > >> > > The moment one of the consumer (for example: consumer
> number
> > > 54)
> > > > > > from
> > > > > > >> the
> > > > > > >> > > first 300 members go out of the group (crossed session
> > > timeout),
> > > > > it
> > > > > > >> (the
> > > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> > the
> > > > > > >> consumer
> > > > > > >> > > member 54 to one of the spare hosts.
> > > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > > being a
> > > > > > >> part
> > > > > > >> > of
> > > > > > >> > > the spare pool.
> > > > > > >> > > This enables us to have lower session timeouts and low
> > latency
> > > > > > >> mirroring,
> > > > > > >> > > in cases where the service providers are OK with having
> > spare
> > > > > hosts.
> > > > > > >> > > This would mean that we would tolerate n consumer members
> > > > leaving
> > > > > > and
> > > > > > >> > > rejoining the group and still provide low latency as long
> > as n
> > > > <=
> > > > > > >> number
> > > > > > >> > of
> > > > > > >> > > spare consumers.
> > > > > > >> > > If there are no spare host available, we can get back to
> the
> > > > idea
> > > > > as
> > > > > > >> > > described in the KIP.
> > > > > > >> > Great idea! In fact on top of static membership we could
> later
> > > > > > introduce
> > > > > > >> > APIs to set hard-coded
> > > > > > >> > client ids to the group and replace the dead host, or as you
> > > > > proposed
> > > > > > to
> > > > > > >> > define spare host as
> > > > > > >> > what I understood as hot backup. I will put both Jason and
> > your
> > > > > > >> > suggestions into a separate section
> > > > > > >> > called "Future works". Note that this spare host idea may be
> > > also
> > > > > > >> solvable
> > > > > > >> > through rebalance protocol
> > > > > > >> > IMO.
> > > > > > >> >
> > > > > > >> > Thank you again for the great feedback!
> > > > > > >> >
> > > > > > >> > Boyang
> > > > > > >> > ________________________________
> > > > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > > > >> > To: dev@kafka.apache.org
> > > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > by
> > > > > > >> > specifying member id
> > > > > > >> >
> > > > > > >> > Hey Dong, sorry for missing your message. I couldn't find
> your
> > > > email
> > > > > > on
> > > > > > >> my
> > > > > > >> > thread, so I will just do a checklist here!
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > 1) The motivation currently explicitly states that the goal
> is
> > > to
> > > > > > >> improve
> > > > > > >> >
> > > > > > >> > performance for heavy state application. It seems that the
> > > > > motivation
> > > > > > >> can
> > > > > > >> >
> > > > > > >> > be stronger with the following use-case. Currently for
> > > MirrorMaker
> > > > > > >> cluster
> > > > > > >> >
> > > > > > >> > with e.g. 100 MirrorMaker processes, it will take a long
> time
> > to
> > > > > > rolling
> > > > > > >> >
> > > > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > process
> > > > > > restart
> > > > > > >> >
> > > > > > >> > will trigger a rebalance which currently pause the
> consumption
> > > of
> > > > > the
> > > > > > >> all
> > > > > > >> >
> > > > > > >> > partitions of the MirrorMaker cluster. With the change
> stated
> > in
> > > > > this
> > > > > > >> >
> > > > > > >> > patch, as long as a MirrorMaker can restart within the
> > specified
> > > > > > timeout
> > > > > > >> >
> > > > > > >> > (e.g. 2 minutes), then we only need constant number of
> > rebalance
> > > > > (e.g.
> > > > > > >> for
> > > > > > >> >
> > > > > > >> > leader restart) for the entire rolling bounce, which will
> > > > > > significantly
> > > > > > >> >
> > > > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > > > opinion,
> > > > > > >> the
> > > > > > >> >
> > > > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if
> > the
> > > > > > >> consumer
> > > > > > >> >
> > > > > > >> > process can be restarted within soon, which helps
> performance
> > > even
> > > > > if
> > > > > > >> >
> > > > > > >> > overhead of state shuffling for a given process is small.
> > > > > > >> >
> > > > > > >> > I just rephrased this part and added it to the KIP. Thanks
> for
> > > > > making
> > > > > > >> the
> > > > > > >> > motivation more solid!
> > > > > > >> >
> > > > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > > > writeup
> > > > > > >> style
> > > > > > >> > of other KIP (e.g. KIP-98) and list the interface change
> such
> > as
> > > > new
> > > > > > >> > configs (e.g. registration timeout), new request/response,
> new
> > > > > > >> AdminClient
> > > > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > Currently
> > > > > some
> > > > > > of
> > > > > > >> > these are specified in the Proposed Change section which
> makes
> > > it
> > > > a
> > > > > > bit
> > > > > > >> > inconvenient to understand the new interface that will be
> > > exposed
> > > > to
> > > > > > >> user.
> > > > > > >> > Explanation of the current two-phase rebalance protocol
> > probably
> > > > can
> > > > > > be
> > > > > > >> > moved out of public interface section.
> > > > > > >> > This is a great suggestion! I just consolidated all the
> public
> > > API
> > > > > > >> > changes, and the whole KIP
> > > > > > >> > looks much more organized!
> > > > > > >> >
> > > > > > >> > 3) There are currently two version of JoinGroupRequest in
> the
> > > KIP
> > > > > and
> > > > > > >> only
> > > > > > >> > one of them has field memberId. This seems confusing.
> > > > > > >> > Yep, I already found this issue and fixed it.
> > > > > > >> >
> > > > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > > > rebalance
> > > > > > >> could
> > > > > > >> > be helpful here, but we will make a call once we finished
> the
> > > > major
> > > > > > >> > implementation". So this seems to be still an open question
> in
> > > the
> > > > > > >> current
> > > > > > >> > design. We probably want to agree on this before voting for
> > the
> > > > KIP.
> > > > > > >> > We have finalized the idea that this API is needed.
> > > > > > >> >
> > > > > > >> > 5) The KIP currently adds new config MEMBER_NAME for
> consumer.
> > > Can
> > > > > you
> > > > > > >> > specify the name of the config key and the default config
> > value?
> > > > > > >> Possible
> > > > > > >> > default values include empty string or null (similar to
> > > > > > transaction.id<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=qWkxpqJMF6ugtVKhupIthbcmSNFmp4sX5EfgvKAiAQo%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > in
> > > > > > >> > producer config).
> > > > > > >> > I have defined the `member.name` in "New configuration"
> > > section.
> > > > > > >> >
> > > > > > >> > 6) Regarding the use of the topic "static_member_map" to
> > persist
> > > > > > member
> > > > > > >> > name map, currently if consumer coordinator broker goes
> > offline,
> > > > > > >> rebalance
> > > > > > >> > is triggered and consumers will try connect to the new
> > > > coordinator.
> > > > > If
> > > > > > >> > these consumers can connect to the new coordinator within
> > > > > > >> > max.poll.interval.ms<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > which by default is 5 minutes, given that broker can
> > > > > > >> > use a deterministic algorithm to determine the partition ->
> > > > > > member_name
> > > > > > >> > mapping, each consumer should get assigned the same set of
> > > > > partitions
> > > > > > >> > without requiring state shuffling. So it is not clear
> whether
> > we
> > > > > have
> > > > > > a
> > > > > > >> > strong use-case for this new logic. Can you help clarify
> what
> > is
> > > > the
> > > > > > >> > benefit of using topic "static_member_map" to persist member
> > > name
> > > > > map?
> > > > > > >> > I have discussed with Guozhang offline, and I believe
> reusing
> > > the
> > > > > > >> current
> > > > > > >> > `_consumer_offsets`
> > > > > > >> > topic is a better and unified solution.
> > > > > > >> >
> > > > > > >> > 7) Regarding the introduction of the expensionTimeoutMs
> > config,
> > > it
> > > > > is
> > > > > > >> > mentioned that "we are using expansion timeout to replace
> > > > rebalance
> > > > > > >> > timeout, which is configured by max.poll.intervals from
> client
> > > > side,
> > > > > > and
> > > > > > >> > using registration timeout to replace session timeout".
> > > Currently
> > > > > the
> > > > > > >> > default max.poll.interval.ms<
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > is configured to be 5 minutes and there will
> > > > > > >> > be only one rebalance if all new consumers can join within 5
> > > > > minutes.
> > > > > > >> So it
> > > > > > >> > is not clear whether we have a strong use-case for this new
> > > > config.
> > > > > > Can
> > > > > > >> you
> > > > > > >> > explain what is the benefit of introducing this new config?
> > > > > > >> > Previously our goal is to use expansion timeout as a
> > workaround
> > > > for
> > > > > > >> > triggering multiple
> > > > > > >> > rebalances when scaling up members are not joining at the
> same
> > > > time.
> > > > > > It
> > > > > > >> is
> > > > > > >> > decided to
> > > > > > >> > be addressed by client side protocol change, so we will not
> > > > > introduce
> > > > > > >> > expansion timeout.
> > > > > > >> >
> > > > > > >> > 8) It is mentioned that "To distinguish between previous
> > version
> > > > of
> > > > > > >> > protocol, we will also increase the join group request
> version
> > > to
> > > > v4
> > > > > > >> when
> > > > > > >> > MEMBER_NAME is set" and "If the broker version is not the
> > latest
> > > > (<
> > > > > > v4),
> > > > > > >> > the join group request shall be downgraded to v3 without
> > setting
> > > > the
> > > > > > >> member
> > > > > > >> > Id". It is probably simpler to just say that this feature is
> > > > enabled
> > > > > > if
> > > > > > >> > JoinGroupRequest V4 is supported on both client and broker
> and
> > > > > > >> MEMBER_NAME
> > > > > > >> > is configured with non-empty string.
> > > > > > >> > Yep, addressed this!
> > > > > > >> >
> > > > > > >> > 9) It is mentioned that broker may return
> > > > NO_STATIC_MEMBER_INFO_SET
> > > > > > >> error
> > > > > > >> > in OffsetCommitResponse for "commit requests under static
> > > > > membership".
> > > > > > >> Can
> > > > > > >> > you clarify how broker determines whether the commit request
> > is
> > > > > under
> > > > > > >> > static membership?
> > > > > > >> >
> > > > > > >> > We have agreed that commit request shouldn't be affected by
> > the
> > > > new
> > > > > > >> > membership, thus
> > > > > > >> > removing it here. Thanks for catching this!
> > > > > > >> >
> > > > > > >> > Let me know if you have further suggestions or concerns.
> Thank
> > > you
> > > > > for
> > > > > > >> > your valuable feedback
> > > > > > >> > to help me design the KIP better! (And I will try to address
> > > your
> > > > > > >> > feedbacks in next round Mayuresh ??)
> > > > > > >> >
> > > > > > >> > Best,
> > > > > > >> > Boyang
> > > > > > >> > ________________________________
> > > > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > > > >> > To: dev@kafka.apache.org
> > > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > by
> > > > > > >> > specifying member id
> > > > > > >> >
> > > > > > >> > Hi Boyang,
> > > > > > >> >
> > > > > > >> > Thanks for updating the KIP. This is a step good direction
> for
> > > > > > stateful
> > > > > > >> > applications and also mirroring applications whose latency
> is
> > > > > affected
> > > > > > >> due
> > > > > > >> > to the rebalance issues that we have today.
> > > > > > >> >
> > > > > > >> > I had a few questions on the current version of the KIP :
> > > > > > >> > For the effectiveness of the KIP, consumer with member.name
> > set
> > > > > will
> > > > > > >> *not
> > > > > > >> > send leave group request* when they go offline
> > > > > > >> >
> > > > > > >> > > By this you mean, even if the application has not called
> > > > > > >> > > KafkaConsumer.poll() within session timeout, it will not
> be
> > > > > sending
> > > > > > >> the
> > > > > > >> > > LeaveGroup request, right?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > > > member.id
> > > > > > }
> > > > > > >> to
> > > > > > >> > track member uniqueness.
> > > > > > >> >
> > > > > > >> > > When is the member.name removed from this map?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Member.id must be set if the *member.name <
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > > > > >> >
> > > > > > >> > *is already
> > > > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > > > >> >
> > > > > > >> > > How is this case handled on the client side? What is the
> > > > > application
> > > > > > >> that
> > > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Session timeout is the timeout we will trigger rebalance
> when
> > a
> > > > > member
> > > > > > >> goes
> > > > > > >> > offline for too long (not sending heartbeat request). To
> make
> > > > static
> > > > > > >> > membership effective, we should increase the default max
> > session
> > > > > > >> timeout to
> > > > > > >> > 30 min so that end user could config it freely.
> > > > > > >> >
> > > > > > >> > > This would mean that it might take more time to detect
> > unowned
> > > > > topic
> > > > > > >> > > partitions and may cause delay for applications that
> perform
> > > > data
> > > > > > >> > mirroring
> > > > > > >> > > tasks. I discussed this with our sre and we have a
> > suggestion
> > > to
> > > > > > make
> > > > > > >> > here
> > > > > > >> > > as listed below separately.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Currently there is a config called *rebalance timeout* which
> > is
> > > > > > >> configured
> > > > > > >> > by consumer *max.poll.intervals*. The reason we set it to
> poll
> > > > > > interval
> > > > > > >> is
> > > > > > >> > because consumer could only send request within the call of
> > > poll()
> > > > > and
> > > > > > >> we
> > > > > > >> > want to wait sufficient time for the join group request.
> When
> > > > > reaching
> > > > > > >> > rebalance timeout, the group will move towards
> > > completingRebalance
> > > > > > stage
> > > > > > >> > and remove unjoined groups
> > > > > > >> >
> > > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Currently there is a config called *rebalance timeout* which
> > is
> > > > > > >> configured
> > > > > > >> > by consumer *max.poll.intervals*. The reason we set it to
> poll
> > > > > > interval
> > > > > > >> is
> > > > > > >> > because consumer could only send request within the call of
> > > poll()
> > > > > and
> > > > > > >> we
> > > > > > >> > want to wait sufficient time for the join group request.
> When
> > > > > reaching
> > > > > > >> > rebalance timeout, the group will move towards
> > > completingRebalance
> > > > > > stage
> > > > > > >> > and remove unjoined groups. This is actually conflicting
> with
> > > the
> > > > > > >> design of
> > > > > > >> > static membership, because those temporarily unavailable
> > members
> > > > > will
> > > > > > >> > potentially reattempt the join group and trigger extra
> > > rebalances.
> > > > > > >> > Internally we would optimize this logic by having rebalance
> > > > timeout
> > > > > > >> only in
> > > > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > > > >> non-responsive
> > > > > > >> > members immediately.
> > > > > > >> >
> > > > > > >> > > What do you mean by " Internally we would optimize this
> > logic
> > > by
> > > > > > >> having
> > > > > > >> > > rebalance timeout only in charge of stopping prepare
> > rebalance
> > > > > > stage,
> > > > > > >> > > without removing non-responsive members immediately."
> There
> > > > would
> > > > > > not
> > > > > > >> be
> > > > > > >> > a
> > > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > > request
> > > > > > later,
> > > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Scale Up
> > > > > > >> >
> > > > > > >> > > The KIP talks about scale up scenario but its not quite
> > clear
> > > > how
> > > > > we
> > > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> > we
> > > > > adding
> > > > > > >> > status
> > > > > > >> > > "learner" ?. Can you shed more light on how this is
> handled
> > in
> > > > the
> > > > > > >> KIP,
> > > > > > >> > if
> > > > > > >> > > its handled?
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > *Discussion*
> > > > > > >> > Larger session timeouts causing latency rise for getting
> data
> > > for
> > > > > > >> un-owned
> > > > > > >> > topic partitions :
> > > > > > >> >
> > > > > > >> > > I think Jason had brought this up earlier about having a
> way
> > > to
> > > > > say
> > > > > > >> how
> > > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > > consumer
> > > > > > >> group.
> > > > > > >> > > If we can do this, then in case of mirroring applications
> we
> > > can
> > > > > do
> > > > > > >> this
> > > > > > >> > :
> > > > > > >> > > Lets say we have a mirroring application that consumes
> from
> > > > Kafka
> > > > > > >> cluster
> > > > > > >> > > A and produces to Kafka cluster B.
> > > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > > Kafka
> > > > > > >> service
> > > > > > >> > > providers can set a mirroring group saying that it will
> > take,
> > > > for
> > > > > > >> example
> > > > > > >> > > 300 consumer hosts/members to achieve the desired
> throughput
> > > and
> > > > > > >> latency
> > > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > > spare
> > > > > in
> > > > > > >> the
> > > > > > >> > > same group.
> > > > > > >> > > So when the first 300 members/consumers to join the group
> > will
> > > > > start
> > > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster
> B.
> > > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > > >> > > The moment one of the consumer (for example: consumer
> number
> > > 54)
> > > > > > from
> > > > > > >> the
> > > > > > >> > > first 300 members go out of the group (crossed session
> > > timeout),
> > > > > it
> > > > > > >> (the
> > > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> > the
> > > > > > >> consumer
> > > > > > >> > > member 54 to one of the spare hosts.
> > > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > > being a
> > > > > > >> part
> > > > > > >> > of
> > > > > > >> > > the spare pool.
> > > > > > >> > > This enables us to have lower session timeouts and low
> > latency
> > > > > > >> mirroring,
> > > > > > >> > > in cases where the service providers are OK with having
> > spare
> > > > > hosts.
> > > > > > >> > > This would mean that we would tolerate n consumer members
> > > > leaving
> > > > > > and
> > > > > > >> > > rejoining the group and still provide low latency as long
> > as n
> > > > <=
> > > > > > >> number
> > > > > > >> > of
> > > > > > >> > > spare consumers.
> > > > > > >> > > If there are no spare host available, we can get back to
> the
> > > > idea
> > > > > as
> > > > > > >> > > described in the KIP.
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> > Thanks,
> > > > > > >> >
> > > > > > >> > Mayuresh
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > > > >> > konstantine@confluent.io> wrote:
> > > > > > >> >
> > > > > > >> > > Hi Boyang.
> > > > > > >> > >
> > > > > > >> > > Thanks for preparing this KIP! It is making good progress
> > and
> > > > will
> > > > > > be
> > > > > > >> a
> > > > > > >> > > great improvement for stateful Kafka applications.
> > > > > > >> > >
> > > > > > >> > > Apologies for my late reply, I was away for a while. Lots
> of
> > > > great
> > > > > > >> > comments
> > > > > > >> > > so far, so I'll probably second most of them in what I
> > suggest
> > > > > below
> > > > > > >> at
> > > > > > >> > > this point.
> > > > > > >> > >
> > > > > > >> > > When I first read the KIP, I wanted to start at the end
> with
> > > > > > something
> > > > > > >> > that
> > > > > > >> > > wasn't highlighted a lot. That was the topic related to
> > > handling
> > > > > > >> > duplicate
> > > > > > >> > > members. I see now that the initial suggestion of handling
> > > this
> > > > > > >> situation
> > > > > > >> > > during offset commit has been removed, and I agree with
> > that.
> > > > > Issues
> > > > > > >> > > related to membership seem to be handled better when the
> > > member
> > > > > > joins
> > > > > > >> the
> > > > > > >> > > group rather than when it tries to commit offsets. This
> also
> > > > > > >> simplifies
> > > > > > >> > how
> > > > > > >> > > many request types need to change in order to incorporate
> > the
> > > > new
> > > > > > >> member
> > > > > > >> > > name field.
> > > > > > >> > >
> > > > > > >> > > I also agree with what Jason and Guozhang have said
> > regarding
> > > > > > >> timeouts.
> > > > > > >> > > Although semantically, it's easier to think of every
> > operation
> > > > > > having
> > > > > > >> its
> > > > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > > > >> consolidation
> > > > > > >> > > seems preferable here. The definition of embedded
> protocols
> > on
> > > > top
> > > > > > of
> > > > > > >> the
> > > > > > >> > > base group membership protocol for rebalancing gives
> enough
> > > > > > >> flexibility
> > > > > > >> > to
> > > > > > >> > > address such needs in each client component separately.
> > > > > > >> > >
> > > > > > >> > > Finally, some minor comments:
> > > > > > >> > > In a few places the new/proposed changes are referred to
> as
> > > > > > "current".
> > > > > > >> > > Which is a bit confusing considering that there is a
> > protocol
> > > in
> > > > > > place
> > > > > > >> > > already, and by "current" someone might understand the
> > > existing
> > > > > one.
> > > > > > >> I'd
> > > > > > >> > > recommend using new/proposed or equivalent when referring
> to
> > > > > changes
> > > > > > >> > > introduced with KIP-345 and current/existing or equivalent
> > > when
> > > > > > >> referring
> > > > > > >> > > to existing behavior.
> > > > > > >> > >
> > > > > > >> > > There's the following sentence in the "Public Interfaces"
> > > > section:
> > > > > > >> > > "Since for many stateful consumer/stream applications, the
> > > state
> > > > > > >> > shuffling
> > > > > > >> > > is more painful than short time partial unavailability."
> > > > > > >> > > However, my understanding is that the changes proposed
> with
> > > > > KIP-345
> > > > > > >> will
> > > > > > >> > > not exploit any partial availability. A suggestion for
> > dealing
> > > > > with
> > > > > > >> > > temporary imbalances has been made in "Incremental
> > Cooperative
> > > > > > >> > Rebalancing"
> > > > > > >> > > which can work well with KIP-345, but here I don't see
> > > proposed
> > > > > > >> changes
> > > > > > >> > > that suggest that some resources (e.g. partitions) will
> keep
> > > > being
> > > > > > >> used
> > > > > > >> > > while others will not be utilized. Thus, you might want to
> > > > adjust
> > > > > > this
> > > > > > >> > > sentence. Correct me if I'm missing something related to
> > that.
> > > > > > >> > >
> > > > > > >> > > In the rejected alternatives, under point 2) I read "we
> can
> > > copy
> > > > > the
> > > > > > >> > member
> > > > > > >> > > id to the config files". I believe it means to say "member
> > > name"
> > > > > > >> unless
> > > > > > >> > I'm
> > > > > > >> > > missing something about reusing member ids. Also below I
> > read:
> > > > "By
> > > > > > >> > allowing
> > > > > > >> > > consumers to optionally specifying a member id" which
> > probably
> > > > > > implies
> > > > > > >> > > "member name" again. In a sense this section highlights a
> > > > > potential
> > > > > > >> > > confusion between member name and member id. I wonder if
> we
> > > > could
> > > > > > >> come up
> > > > > > >> > > with a better term for the new field. StaticTag,
> > StaticLabel,
> > > or
> > > > > > even
> > > > > > >> > > StaticName are some suggestions that could potentially
> help
> > > with
> > > > > > >> > confusion
> > > > > > >> > > between MemberId and MemberName and what corresponds to
> > what.
> > > > But
> > > > > I
> > > > > > >> > > wouldn't like to disrupt the discussion with naming
> > > conventions
> > > > > too
> > > > > > >> much
> > > > > > >> > at
> > > > > > >> > > this point. I just mention it here as a thought.
> > > > > > >> > >
> > > > > > >> > > Looking forward to see the final details of this KIP.
> Great
> > > work
> > > > > so
> > > > > > >> far!
> > > > > > >> > >
> > > > > > >> > > Konstantine
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > > > bchen11@outlook.com>
> > > > > > >> wrote:
> > > > > > >> > >
> > > > > > >> > > > Thanks Guozhang for the great summary here, and I have
> > been
> > > > > > >> following
> > > > > > >> > up
> > > > > > >> > > > the action items here.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > > > timeout
> > > > > > and
> > > > > > >> > > > registration timeout. Great to see them being addressed
> in
> > > > > client
> > > > > > >> side!
> > > > > > >> > > >   2.  I double checked the design and I believe that it
> is
> > > ok
> > > > to
> > > > > > >> have
> > > > > > >> > > both
> > > > > > >> > > > static member and dynamic member co-exist in the same
> > group.
> > > > So
> > > > > > the
> > > > > > >> > > upgrade
> > > > > > >> > > > shouldn't be destructive and we are removing the two
> > > > membership
> > > > > > >> > protocol
> > > > > > >> > > > switching APIs.
> > > > > > >> > > >   3.  I only have question about this one. I'm still
> > reading
> > > > the
> > > > > > >> > > KafkaApis
> > > > > > >> > > > code here. Should I just use the same authorization
> logic
> > > for
> > > > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > > > >> > > >   4.  I'm very excited to see this work with K8! Like
> you
> > > > > > suggested,
> > > > > > >> > this
> > > > > > >> > > > feature could be better addressed in a separate KIP
> > because
> > > it
> > > > > is
> > > > > > >> > pretty
> > > > > > >> > > > independent. I could start drafting the KIP once the
> > current
> > > > > > >> proposal
> > > > > > >> > is
> > > > > > >> > > > approved.
> > > > > > >> > > >   5.  I believe that we don't need fencing in offset
> > commit
> > > > > > request,
> > > > > > >> > > since
> > > > > > >> > > > duplicate member.name issue could be handled by join
> > group
> > > > > > >> request. We
> > > > > > >> > > > shall reject join group with known member name but no
> > member
> > > > id
> > > > > > >> (which
> > > > > > >> > > > means we already have an active member using this
> > identity).
> > > > > > >> > > >   6.  I agree to remove that internal config once we
> move
> > > > > forward
> > > > > > >> with
> > > > > > >> > > > static membership. And I already removed the entire
> > section
> > > > from
> > > > > > the
> > > > > > >> > KIP.
> > > > > > >> > > >
> > > > > > >> > > > Let me know if you have other concerns.
> > > > > > >> > > >
> > > > > > >> > > > Best,
> > > > > > >> > > > Boyang
> > > > > > >> > > > ________________________________
> > > > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > > > >> > > > To: dev
> > > > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > > > rebalances
> > > > > > >> by
> > > > > > >> > > > specifying member id
> > > > > > >> > > >
> > > > > > >> > > > Hello Boyang,
> > > > > > >> > > >
> > > > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > > > appreciate
> > > > > > >> your
> > > > > > >> > > > patience answering to the feedbacks from the community.
> > I'd
> > > > like
> > > > > > to
> > > > > > >> add
> > > > > > >> > > my
> > > > > > >> > > > 2cents here:
> > > > > > >> > > >
> > > > > > >> > > > 1. By introducing another two timeout configs,
> > > > > > registration_timeout
> > > > > > >> and
> > > > > > >> > > > expansion_timeout, we are effectively having four
> timeout
> > > > > configs:
> > > > > > >> > > session
> > > > > > >> > > > timeout, rebalance timeout (configured as "
> > > > max.poll.interval.ms
> > > > > "
> > > > > > on
> > > > > > >> > > client
> > > > > > >> > > > side), and these two. Interplaying these timeout configs
> > can
> > > > be
> > > > > > >> quite
> > > > > > >> > > hard
> > > > > > >> > > > for users with such complexity, and hence I'm wondering
> if
> > > we
> > > > > can
> > > > > > >> > > simplify
> > > > > > >> > > > the situation with as less possible timeout configs as
> > > > possible.
> > > > > > >> Here
> > > > > > >> > is
> > > > > > >> > > a
> > > > > > >> > > > concrete suggestion I'd like propose:
> > > > > > >> > > >
> > > > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> > > addition
> > > > > to
> > > > > > >> the
> > > > > > >> > > > session_timeout for static members, we can just reuse
> the
> > > > > > >> > session_timeout
> > > > > > >> > > > and ask users to set it to a larger value when they are
> > > > > upgrading
> > > > > > a
> > > > > > >> > > dynamic
> > > > > > >> > > > client to a static client by setting the "member.name"
> at
> > > the
> > > > > > same
> > > > > > >> > time.
> > > > > > >> > > > By
> > > > > > >> > > > default, the broker-side min.session.timeout is 6
> seconds
> > > and
> > > > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable
> > to
> > > me
> > > > > (we
> > > > > > >> can
> > > > > > >> > of
> > > > > > >> > > > course modify this broker config to enlarge the valid
> > > interval
> > > > > if
> > > > > > we
> > > > > > >> > want
> > > > > > >> > > > in practice). And then we should also consider removing
> > the
> > > > > > >> condition
> > > > > > >> > for
> > > > > > >> > > > marking a client as failed if the rebalance timeout has
> > > > reached
> > > > > > >> while
> > > > > > >> > the
> > > > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > > > session_timeout
> > > > > > >> > and
> > > > > > >> > > > rebalance_timeout are totally separated: the former is
> > only
> > > > used
> > > > > > to
> > > > > > >> > > > determine if a consumer member of the group should be
> > marked
> > > > as
> > > > > > >> failed
> > > > > > >> > > and
> > > > > > >> > > > kicked out of the group, and the latter is only used to
> > > > > determine
> > > > > > >> the
> > > > > > >> > > > longest time coordinator should wait for
> PREPARE_REBALANCE
> > > > > phase.
> > > > > > In
> > > > > > >> > > other
> > > > > > >> > > > words if a member did not send the JoinGroup in time of
> > the
> > > > > > >> > > > rebalance_timeout, we still include it in the new
> > generation
> > > > of
> > > > > > the
> > > > > > >> > group
> > > > > > >> > > > and use its old subscription info to send to leader for
> > > > > > assignment.
> > > > > > >> > Later
> > > > > > >> > > > if the member came back with HeartBeat request, we can
> > still
> > > > > > follow
> > > > > > >> the
> > > > > > >> > > > normal path to bring it to the latest generation while
> > > > checking
> > > > > > that
> > > > > > >> > its
> > > > > > >> > > > sent JoinGroup request contains the same subscription
> info
> > > as
> > > > we
> > > > > > >> used
> > > > > > >> > to
> > > > > > >> > > > assign the partitions previously (which should be likely
> > the
> > > > > case
> > > > > > in
> > > > > > >> > > > practice). In addition, we should let static members to
> > not
> > > > send
> > > > > > the
> > > > > > >> > > > LeaveGroup request when it is gracefully shutdown, so
> > that a
> > > > > > static
> > > > > > >> > > member
> > > > > > >> > > > can only be leaving the group if its session has timed
> > out,
> > > OR
> > > > > it
> > > > > > >> has
> > > > > > >> > > been
> > > > > > >> > > > indicated to not exist in the group any more (details
> > > below).
> > > > > > >> > > >
> > > > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > > > Cooperative
> > > > > > >> > > > Rebalancing, in which we will encode the "when to
> > rebalance"
> > > > > logic
> > > > > > >> at
> > > > > > >> > the
> > > > > > >> > > > application level, instead of at the protocol level. By
> > > doing
> > > > > this
> > > > > > >> we
> > > > > > >> > can
> > > > > > >> > > > also enable a few other optimizations, e.g. at the
> Streams
> > > > level
> > > > > > to
> > > > > > >> > first
> > > > > > >> > > > build up the state store as standby tasks and then
> > trigger a
> > > > > > second
> > > > > > >> > > > rebalance to actually migrate the active tasks while
> > keeping
> > > > the
> > > > > > >> actual
> > > > > > >> > > > rebalance latency and hence unavailability window to be
> > > small
> > > > (
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=N8BTmhWAmDGJc9%2BQl6ulM9Qa5vzxIyXaGzCDILSIehs%3D&amp;reserved=0
> > > > > > >> > > ).
> > > > > > >> > > > I'd propose we align
> > > > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > > > >> > expansion_timeout
> > > > > > >> > > as
> > > > > > >> > > > part of the protocol layer, but only do that at the
> > > > > application's
> > > > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We
> > can
> > > > > > still,
> > > > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > > > >> > > > <
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=VO0%2F5TczxUBuJkK7NurBwa1X0wIXwm0WHx4jrCzY0%2Fo%3D&amp;reserved=0
> > > > > > >> > > >*"
> > > > > > >> > > > though as part of this KIP
> > > > > > >> > > > since we have discussed about its limit and think it is
> > > > actually
> > > > > > >> not a
> > > > > > >> > > very
> > > > > > >> > > > good design and could be replaced with client-side logic
> > > > above.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> > > this
> > > > > KIP.
> > > > > > >> More
> > > > > > >> > > > specifically, let's say after we have upgraded broker
> > > version
> > > > to
> > > > > > be
> > > > > > >> > able
> > > > > > >> > > to
> > > > > > >> > > > recognize the new versions of JoinGroup request and the
> > > admin
> > > > > > >> requests,
> > > > > > >> > > how
> > > > > > >> > > > should we upgrade the clients and enable static groups?
> On
> > > top
> > > > > of
> > > > > > my
> > > > > > >> > head
> > > > > > >> > > > if we do a rolling bounce in which we set the
> member.name
> > > > > config
> > > > > > as
> > > > > > >> > well
> > > > > > >> > > > as
> > > > > > >> > > > optionally increase the session.timeout config when we
> > > bounce
> > > > > each
> > > > > > >> > > > instance, then during this rolling bounces we will have
> a
> > > > group
> > > > > > >> > contained
> > > > > > >> > > > with both dynamic members and static members. It means
> > that
> > > we
> > > > > > >> should
> > > > > > >> > > have
> > > > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > > > JoinGroup
> > > > > > >> > > requests
> > > > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > > > member.id"
> > > > > > >> > > mapping
> > > > > > >> > > > will only be partial at this scenario. Also could you
> > > describe
> > > > > if
> > > > > > >> the
> > > > > > >> > > > upgrade to the first version that support this feature
> > would
> > > > > ever
> > > > > > >> get
> > > > > > >> > any
> > > > > > >> > > > benefits, or only the future upgrade path for rolling
> > > bounces
> > > > > > could
> > > > > > >> get
> > > > > > >> > > > benefits out of this feature?
> > > > > > >> > > >
> > > > > > >> > > > If that's the case and we will do 1) as suggested above,
> > do
> > > we
> > > > > > still
> > > > > > >> > need
> > > > > > >> > > > the enableStaticMembership and enableDynamicMembership
> > admin
> > > > > > >> requests
> > > > > > >> > any
> > > > > > >> > > > more? Seems it is not necessary any more as we will only
> > > have
> > > > > the
> > > > > > >> > notion
> > > > > > >> > > of
> > > > > > >> > > > "dynamic or static members" that can co-exist in a group
> > > while
> > > > > > >> there no
> > > > > > >> > > > notion of "dynamic or static groups", and hence these
> two
> > > > > requests
> > > > > > >> are
> > > > > > >> > > not
> > > > > > >> > > > needed anymore.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 3. We need to briefly talk about the implications for
> ACL
> > as
> > > > we
> > > > > > >> > introduce
> > > > > > >> > > > new admin requests that are related to a specific
> > group.id.
> > > > For
> > > > > > >> > example,
> > > > > > >> > > > we
> > > > > > >> > > > need to make sure that whoever created the group or
> joined
> > > the
> > > > > > group
> > > > > > >> > can
> > > > > > >> > > > actually send admin requests for the group, otherwise
> the
> > > > > > >> application
> > > > > > >> > > > owners need to bother the Kafka operators on a
> > multi-tenant
> > > > > > cluster
> > > > > > >> > every
> > > > > > >> > > > time they want to send any admin requests for their
> groups
> > > > which
> > > > > > >> would
> > > > > > >> > be
> > > > > > >> > > > an operational nightmare.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > 4. I like Jason's suggestion of adding an optional field
> > for
> > > > the
> > > > > > >> list
> > > > > > >> > of
> > > > > > >> > > > member names, and I'm wondering if that can be done as
> > part
> > > of
> > > > > the
> > > > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > > > members,
> > > > > > we
> > > > > > >> > will
> > > > > > >> > > > enforce a rebalance immediately since it indicates that
> > some
> > > > > > static
> > > > > > >> > > member
> > > > > > >> > > > will be officially kicked out of the group and some new
> > > static
> > > > > > >> members
> > > > > > >> > > may
> > > > > > >> > > > be added. So back to 1.a) above, a static member can
> only
> > be
> > > > > > kicked
> > > > > > >> out
> > > > > > >> > > of
> > > > > > >> > > > the group if a) its session (arguably long period of
> time)
> > > has
> > > > > > timed
> > > > > > >> > out,
> > > > > > >> > > > and b) this admin request explicitly state that it is no
> > > > longer
> > > > > > >> part of
> > > > > > >> > > the
> > > > > > >> > > > group. As for execution I'm fine with keeping it as a
> > future
> > > > > work
> > > > > > of
> > > > > > >> > this
> > > > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > > > >> > > >
> > > > > > >> > > > Following are minor comments:
> > > > > > >> > > >
> > > > > > >> > > > 5. I'm not sure if we need to include "member.name" as
> > part
> > > > of
> > > > > > the
> > > > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > > > memberId
> > > > > > >> plus
> > > > > > >> > > the
> > > > > > >> > > > generation number should be sufficient for fencing even
> > with
> > > > > > static
> > > > > > >> > > > members.
> > > > > > >> > > >
> > > > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get
> rid
> > > of
> > > > > the
> > > > > > "
> > > > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > Guozhang
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> > > lindong28@gmail.com
> > > > >
> > > > > > >> wrote:
> > > > > > >> > > >
> > > > > > >> > > > > Hey Boyang,
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks for the proposal! This is very useful. I have
> > some
> > > > > > comments
> > > > > > >> > > below:
> > > > > > >> > > > >
> > > > > > >> > > > > 1) The motivation currently explicitly states that the
> > > goal
> > > > is
> > > > > > to
> > > > > > >> > > improve
> > > > > > >> > > > > performance for heavy state application. It seems that
> > the
> > > > > > >> motivation
> > > > > > >> > > can
> > > > > > >> > > > > be stronger with the following use-case. Currently for
> > > > > > MirrorMaker
> > > > > > >> > > > cluster
> > > > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a
> long
> > > > time
> > > > > to
> > > > > > >> > > rolling
> > > > > > >> > > > > bounce the entire MirrorMaker cluster. Each
> MirrorMaker
> > > > > process
> > > > > > >> > restart
> > > > > > >> > > > > will trigger a rebalance which currently pause the
> > > > consumption
> > > > > > of
> > > > > > >> the
> > > > > > >> > > all
> > > > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > > > stated
> > > > > in
> > > > > > >> this
> > > > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > > > specified
> > > > > > >> > > timeout
> > > > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > > > rebalance
> > > > > > >> > (e.g.
> > > > > > >> > > > for
> > > > > > >> > > > > leader restart) for the entire rolling bounce, which
> > will
> > > > > > >> > significantly
> > > > > > >> > > > > improves the availability of the MirrorMaker pipeline.
> > In
> > > my
> > > > > > >> opinion,
> > > > > > >> > > the
> > > > > > >> > > > > main benefit of the KIP is to avoid unnecessary
> > rebalance
> > > if
> > > > > the
> > > > > > >> > > consumer
> > > > > > >> > > > > process can be restarted within soon, which helps
> > > > performance
> > > > > > >> even if
> > > > > > >> > > > > overhead of state shuffling for a given process is
> > small.
> > > > > > >> > > > >
> > > > > > >> > > > > 2) In order to simplify the KIP reading, can you
> follow
> > > the
> > > > > > >> writeup
> > > > > > >> > > style
> > > > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface
> change
> > > > such
> > > > > as
> > > > > > >> new
> > > > > > >> > > > > configs (e.g. registration timeout), new
> > request/response,
> > > > new
> > > > > > >> > > > AdminClient
> > > > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > > > Currently
> > > > > > >> some
> > > > > > >> > > of
> > > > > > >> > > > > these are specified in the Proposed Change section
> which
> > > > makes
> > > > > > it
> > > > > > >> a
> > > > > > >> > bit
> > > > > > >> > > > > inconvenient to understand the new interface that will
> > be
> > > > > > exposed
> > > > > > >> to
> > > > > > >> > > > user.
> > > > > > >> > > > > Explanation of the current two-phase rebalance
> protocol
> > > > > probably
> > > > > > >> can
> > > > > > >> > be
> > > > > > >> > > > > moved out of public interface section.
> > > > > > >> > > > >
> > > > > > >> > > > > 3) There are currently two version of JoinGroupRequest
> > in
> > > > the
> > > > > > KIP
> > > > > > >> and
> > > > > > >> > > > only
> > > > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > > > >> > > > >
> > > > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to
> > force
> > > > > > >> rebalance
> > > > > > >> > > could
> > > > > > >> > > > > be helpful here, but we will make a call once we
> > finished
> > > > the
> > > > > > >> major
> > > > > > >> > > > > implementation". So this seems to be still an open
> > > question
> > > > in
> > > > > > the
> > > > > > >> > > > current
> > > > > > >> > > > > design. We probably want to agree on this before
> voting
> > > for
> > > > > the
> > > > > > >> KIP.
> > > > > > >> > > > >
> > > > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > > > consumer.
> > > > > > Can
> > > > > > >> > you
> > > > > > >> > > > > specify the name of the config key and the default
> > config
> > > > > value?
> > > > > > >> > > Possible
> > > > > > >> > > > > default values include empty string or null (similar
> to
> > > > > > >> > transaction.id
> > > > > > >> > > > in
> > > > > > >> > > > > producer config).
> > > > > > >> > > > >
> > > > > > >> > > > > 6) Regarding the use of the topic "static_member_map"
> to
> > > > > persist
> > > > > > >> > member
> > > > > > >> > > > > name map, currently if consumer coordinator broker
> goes
> > > > > offline,
> > > > > > >> > > > rebalance
> > > > > > >> > > > > is triggered and consumers will try connect to the new
> > > > > > >> coordinator.
> > > > > > >> > If
> > > > > > >> > > > > these consumers can connect to the new coordinator
> > within
> > > > > > >> > > > > max.poll.interval.ms which by default is 5 minutes,
> > given
> > > > > that
> > > > > > >> > broker
> > > > > > >> > > > can
> > > > > > >> > > > > use a deterministic algorithm to determine the
> partition
> > > ->
> > > > > > >> > member_name
> > > > > > >> > > > > mapping, each consumer should get assigned the same
> set
> > of
> > > > > > >> partitions
> > > > > > >> > > > > without requiring state shuffling. So it is not clear
> > > > whether
> > > > > we
> > > > > > >> > have a
> > > > > > >> > > > > strong use-case for this new logic. Can you help
> clarify
> > > > what
> > > > > is
> > > > > > >> the
> > > > > > >> > > > > benefit of using topic "static_member_map" to persist
> > > member
> > > > > > name
> > > > > > >> > map?
> > > > > > >> > > > >
> > > > > > >> > > > > 7) Regarding the introduction of the
> expensionTimeoutMs
> > > > > config,
> > > > > > >> it is
> > > > > > >> > > > > mentioned that "we are using expansion timeout to
> > replace
> > > > > > >> rebalance
> > > > > > >> > > > > timeout, which is configured by max.poll.intervals
> from
> > > > client
> > > > > > >> side,
> > > > > > >> > > and
> > > > > > >> > > > > using registration timeout to replace session
> timeout".
> > > > > > Currently
> > > > > > >> the
> > > > > > >> > > > > default max.poll.interval.ms is configured to be 5
> > > minutes
> > > > > and
> > > > > > >> there
> > > > > > >> > > > will
> > > > > > >> > > > > be only one rebalance if all new consumers can join
> > > within 5
> > > > > > >> minutes.
> > > > > > >> > > So
> > > > > > >> > > > it
> > > > > > >> > > > > is not clear whether we have a strong use-case for
> this
> > > new
> > > > > > >> config.
> > > > > > >> > Can
> > > > > > >> > > > you
> > > > > > >> > > > > explain what is the benefit of introducing this new
> > > config?
> > > > > > >> > > > >
> > > > > > >> > > > > 8) It is mentioned that "To distinguish between
> previous
> > > > > version
> > > > > > >> of
> > > > > > >> > > > > protocol, we will also increase the join group request
> > > > version
> > > > > > to
> > > > > > >> v4
> > > > > > >> > > when
> > > > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not
> > the
> > > > > latest
> > > > > > >> (<
> > > > > > >> > > v4),
> > > > > > >> > > > > the join group request shall be downgraded to v3
> without
> > > > > setting
> > > > > > >> the
> > > > > > >> > > > member
> > > > > > >> > > > > Id". It is probably simpler to just say that this
> > feature
> > > is
> > > > > > >> enabled
> > > > > > >> > if
> > > > > > >> > > > > JoinGroupRequest V4 is supported on both client and
> > broker
> > > > and
> > > > > > >> > > > MEMBER_NAME
> > > > > > >> > > > > is configured with non-empty string.
> > > > > > >> > > > >
> > > > > > >> > > > > 9) It is mentioned that broker may return
> > > > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > > > >> > > error
> > > > > > >> > > > > in OffsetCommitResponse for "commit requests under
> > static
> > > > > > >> > membership".
> > > > > > >> > > > Can
> > > > > > >> > > > > you clarify how broker determines whether the commit
> > > request
> > > > > is
> > > > > > >> under
> > > > > > >> > > > > static membership?
> > > > > > >> > > > >
> > > > > > >> > > > > Thanks,
> > > > > > >> > > > > Dong
> > > > > > >> > > > >
> > > > > > >> > > >
> > > > > > >> > > >
> > > > > > >> > > > --
> > > > > > >> > > > -- Guozhang
> > > > > > >> > > >
> > > > > > >> > >
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > --
> > > > > > >> > -Regards,
> > > > > > >> > Mayuresh R. Gharat
> > > > > > >> > (862) 250-7125
> > > > > > >> >
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > > -Regards,
> > > > > > > Mayuresh R. Gharat
> > > > > > > (862) 250-7125
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -Regards,
> > > > > > Mayuresh R. Gharat
> > > > > > (862) 250-7125
> > > > > >
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
> >
> > --
> > -- Guozhang
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
I like Guozhang's suggestion to not have to wait for session timeout in
case we know that we want to downsize the consumer group and redistribute
the partitions among the remaining consumers.
IIUC, with the above suggestions, the admin api
"removeMemberFromGroup(groupId, list[instanceId])" or
"removeMemberFromGroup(groupId, instanceId)", will automatically cause a
rebalance, right?
I would prefer ist[instanceid] because that's more general scenario.

Also I was thinking if we can have a replace API, that takes in a map of
old to new instance Ids. Such that we can replace a consumer.
IF we have this api, and if a consumer host goes down due to hardware
issues, we can have another host spin up and take its place. This is like a
cold backup which can be a step towards providing the hot backup that we
discussed earlier in the KIP.
Thoughts?

Thanks,

Mayuresh

On Thu, Nov 29, 2018 at 1:30 AM Boyang Chen <bc...@outlook.com> wrote:

> In fact I feel that it's more convenient for user to specify a list of
> instance id prefixes. Because
> for general consumer application we couldn't always find a proper prefix
> to remove a list of consumers.
> So we are either adding list[instanceid prefix], or we could add two
> fields: instanceid prefix, and list[instanceid]
> for clarity purpose. As you know, two options are equivalent since full
> name is subset of prefix.
>
> Let me know your thoughts!
>
> Boyang
> ________________________________
> From: Boyang Chen <bc...@outlook.com>
> Sent: Thursday, November 29, 2018 3:39 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Thanks Guozhang for the new proposal here!
>
> So I'd like to propose a slightly modified version of LeaveGroupRequest:
> instead of letting the static member consumer client themselves to send the
> request (which means we still need to have some hidden configs to turn it
> off like we did today), how about just letting any other client to send
> this request since the LeaveGroupRequest only requires group.id and
> member.id? So back to your operational scenarios, if some static member
> has
> been found crashed and it is not likely to comeback, or we simply want to
> shrink the size of the group by shutting down some static members, we can
> use an admin client to send the LeaveGroupRequest after the instance has
> been completely shutdown or crashed to kick them out of the group and also
> triggers the rebalance.
>
> One issue though, is that users may not know the member id required in the
> LeaveGroupRequest. To work around it we can add the `group.instance.id`
> along with the member id as well and then allow member id null-able. The
> coordinator logic would then be modified as 1) if member.id is specified,
> ignore instance.id and always use member.id to find the member to kick
> out,
> 2) otherwise, try with the instance.id to find the corresponding member.id
> and kick it out, 3) if none is found, reject with an error code.
>
> So in sum the alternative changes are:
>
> a) Modify LeaveGroupRequest to add group.instance.id
> b) Modify coordinator logic to handle such request on the broker side.
> c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> instanceId)" which will be translated as a LeaveGroupRequest.
> d) [Optional] we can even batch the request by allowing
> "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> member.id`
> and `instance.id` field of LeaveGroupRequest to be an array instead of a
> single entry.
> e) We can also remove the admin ConsumerRebalanceRequest as well for
> simplicity (why not? paranoid of having as less request protocols as
> possible :), as it is not needed anymore with the above proposal.
> I agree that reusing LeaveGroupRequest is actually a good idea: we only
> need to iterate
> over an existing request format. Also I found that we haven't discussed
> how we want to enable
> this feature on Streaming applications, which is different from common
> consumer application in that
> Stream app uses stream thread as individual consumer.
> For example if user specifies the client id, the stream consumer client id
> will be like:
> User client id + "-StreamThread-" + thread id + "-consumer"
>
> So I'm thinking we should do sth similar for defining group.instance.id
> on Stream. We shall define another
> config called `stream.instance.id` which would be used as prefix, and for
> each thread consumer the formula
> will look like:
> `group.instance.id` = `stream.instance.id` + "-" + thread id + "-consumer"
>
> And for the ease of use, the interface of leave group request could
> include `group.instance.id.prefix` instead of
> `group.instance.id` so that we could batch remove consumers relating to a
> single stream instance. This is more intuitive
> and flexible since specifying names of 16~32 * n (n = number of stream
> instances to shut down) consumers is not an easy
> job without client management tooling.
>
> How does this workaround sound?
>
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Thursday, November 29, 2018 2:38 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> I was thinking that with the optional static members in the admin
> ConsumerRebalanceRequest it should be sufficient to kick out the static
> member before their session timeout (arguably long in practice) have not
> reached. But now I see your concern is that in some situations the admin
> operators may not even know the full list of static members, but ONLY know
> which static member has failed and hence would like to kick out of the
> group.
>
> So I'd like to propose a slightly modified version of LeaveGroupRequest:
> instead of letting the static member consumer client themselves to send the
> request (which means we still need to have some hidden configs to turn it
> off like we did today), how about just letting any other client to send
> this request since the LeaveGroupRequest only requires group.id and
> member.id? So back to your operational scenarios, if some static member
> has
> been found crashed and it is not likely to comeback, or we simply want to
> shrink the size of the group by shutting down some static members, we can
> use an admin client to send the LeaveGroupRequest after the instance has
> been completely shutdown or crashed to kick them out of the group and also
> triggers the rebalance.
>
> One issue though, is that users may not know the member id required in the
> LeaveGroupRequest. To work around it we can add the `group.instance.id`
> along with the member id as well and then allow member id null-able. The
> coordinator logic would then be modified as 1) if member.id is specified,
> ignore instance.id and always use member.id to find the member to kick
> out,
> 2) otherwise, try with the instance.id to find the corresponding member.id
> and kick it out, 3) if none is found, reject with an error code.
>
> So in sum the alternative changes are:
>
> a) Modify LeaveGroupRequest to add group.instance.id
> b) Modify coordinator logic to handle such request on the broker side.
> c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
> instanceId)" which will be translated as a LeaveGroupRequest.
> d) [Optional] we can even batch the request by allowing
> "removeMemberFromGroup(groupId, list[instanceId])" and then make `
> member.id`
> and `instance.id` field of LeaveGroupRequest to be an array instead of a
> single entry.
> e) We can also remove the admin ConsumerRebalanceRequest as well for
> simplicity (why not? paranoid of having as less request protocols as
> possible :), as it is not needed anymore with the above proposal.
>
>
> WDYT?
>
>
> Guozhang
>
> On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Guozhang and Mayuresh for the follow up! Answers are listed below.
> >
> >
> > >  5. Regarding "So in summary, *the member will only be removed due to
> > > session timeout*. We shall remove it from both in-memory static member
> > name
> > > mapping and member list." If the rebalance is invoked manually using
> the
> > > the admin apis, how long should the group coordinator wait for the
> > members
> > > of the group to send a JoinGroupRequest for participating in the
> > rebalance?
> > > How is a lagging consumer handled?
> >
> > Great question. Let's use c1~c4 example here:
> >
> >   1.  Consumer c1, c2, c3, c4 in stable state
> >   2.  c4 goes down and we detect this issue before session timeout
> through
> > client monitoring. Initiate a ConsumerRebalanceRequest.
> >   3.  A rebalance will be kicking off, and after rebalance timeout we
> > shall keep the same assignment for c1~4, if the session timeout for c4
> > hasn't reached
> >   4.  Group back to stable with c1~4 (although c4 is actually offline)
> >   5.  c4 session timeout finally reached: another rebalance triggered.
> >
> > For step 3, if session timeout triggered within rebalance timeout, only
> > c1~3 will be participating in the rebalance. This is what we mean by
> saying
> > "rebalance
> > timeout shall not remove current members, only session timeout will do."
> > As you could see this is not an ideal scenario: we trigger extra
> rebalance
> > at step 5. In my reply to Guozhang I'm asking whether we should still use
> > LeaveGroupRequest for static members to send a signal to broker saying
> "I'm
> > currently offline", and when we send ConsumerRebalanceRequest to broker,
> we
> > will actually kick off c4 because it says it's offline already, saving
> one
> > or multiple additional rebalances later. This way the
> > ConsumerRebalanceRequest will be more effective in making correct
> judgement
> > on the group status since we have more feedback from client side.
> >
> > > - When we say that we would use invokeConsumerRebalance(groupId) to
> down
> > > scale, with the example in the above question, how will the
> > > GroupCoordinator know that c4 should be kicked out of the group since
> we
> > > are trying to invoke rebalance proactively without waiting for c4's
> > session
> > > time out to expire. Should there be a way of telling the
> GroupCoordinator
> > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > Previous proposal should be suffice to answer this question 😊
> >
> > - Also it looks like the statement "If the `member.id` uses
> > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> replace
> > > the one within current map, if `group.member.name` is known. Also once
> > we
> > > are done with KIP-394
> > > <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > > >,
> > > all the join group requests are requiring `member.id` to physically
> > enter
> > > the consumer group. This way the latest joined " is incomplete. Can you
> > > take a look at this?
> > > Also when we say "all the join group requests are requiring `member.id
> `
> > to
> > > physically enter the consumer group." because a newly started consumer
> > will
> > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > assigns a member.id to the newly started consumer, it has to use it
> for
> > > any
> > > future JoinGroupRequests. Is my understanding correct?
> > >
> > Thanks for catching it! And yes, we shall use one extra round-trip
> between
> > consumer
> > and broker to inform the new member id allocation.
> >
> > Next is the replies to Guozhang's comment:
> > 2) I once have a discussion about the LeaveGroupRequest for static
> members,
> > and the reason for not having it for static members is that we'd need to
> > make it a configurable behavior as well (i.e. the likelihood that a
> static
> > member may shutdown but come back later may be even larger than the
> > likelihood that a shutdown static member would not come back), and when a
> > shutdown is complete the instance cannot tell whether or not it will come
> > back by itself. And hence letting a third party (think: admin used by K8s
> > plugins) issuing a request to indicate static member changes would be
> more
> > plausible.
> >
> > I think having an optional list of all the static members that are still
> in
> > the group, rather than the members to be removed since the latter looks a
> > bit less flexible to me, in the request is a good idea (remember we
> allow a
> > group to have both static and dynamic members at the same time, so when
> > receiving the request, we will only do the diff and add / remove the
> static
> > members directly only, while still let the dynamic members to try to
> > re-join the group with the rebalance timeout).
> > I'm also in favor of storing all the in-group static members. In fact we
> > could reuse
> > the static membership mapping to store this information. Do you think
> > that we should let static member send leave group request to indicate
> > their status of "leaving",
> > and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> > suggesting we should
> > remove those members when kicking off rebalance since we are shutting
> them
> > down already.
> >
> > 3) personally I favor "ids" over "names" :) Since we already have some
> > "ids" and hence it sounds more consistent, plus on the producer side we
> > have a `transactional.id` whose semantics is a bit similar to this one,
> > i.e. for unique distinguishment of a client which may comes and goes but
> > need to be persist over multiple "instance life-times".
> > Sure we have enough votes for ids 😊I will finalize the name to `
> > group.instance.id`, does that
> > sound good?
> >
> > Best,
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Wednesday, November 28, 2018 4:51 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Regarding Jason's question and Boyang's responses:
> >
> > 2) I once have a discussion about the LeaveGroupRequest for static
> members,
> > and the reason for not having it for static members is that we'd need to
> > make it a configurable behavior as well (i.e. the likelihood that a
> static
> > member may shutdown but come back later may be even larger than the
> > likelihood that a shutdown static member would not come back), and when a
> > shutdown is complete the instance cannot tell whether or not it will come
> > back by itself. And hence letting a third party (think: admin used by K8s
> > plugins) issuing a request to indicate static member changes would be
> more
> > plausible.
> >
> > I think having an optional list of all the static members that are still
> in
> > the group, rather than the members to be removed since the latter looks a
> > bit less flexible to me, in the request is a good idea (remember we
> allow a
> > group to have both static and dynamic members at the same time, so when
> > receiving the request, we will only do the diff and add / remove the
> static
> > members directly only, while still let the dynamic members to try to
> > re-join the group with the rebalance timeout).
> >
> > 3) personally I favor "ids" over "names" :) Since we already have some
> > "ids" and hence it sounds more consistent, plus on the producer side we
> > have a `transactional.id` whose semantics is a bit similar to this one,
> > i.e. for unique distinguishment of a client which may comes and goes but
> > need to be persist over multiple "instance life-times".
> >
> >
> > Guozhang
> >
> >
> > On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> > gharatmayuresh15@gmail.com>
> > wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks for the replies. Please find the follow up queries below.
> > >
> > >     5. Regarding "So in summary, *the member will only be removed due
> to
> > > session timeout*. We shall remove it from both in-memory static member
> > name
> > > mapping and member list." If the rebalance is invoked manually using
> the
> > > the admin apis, how long should the group coordinator wait for the
> > members
> > > of the group to send a JoinGroupRequest for participating in the
> > rebalance?
> > > How is a lagging consumer handled?
> > > The plan is to disable member kick out when rebalance.timeout is
> reached,
> > > so basically we are not "waiting" any
> > > join group request from existing members; we shall just rebalance base
> on
> > > what we currently have within the group
> > > metadata. Lagging consumer will trigger rebalance later if session
> > timeout
> > > > rebalance timeout.
> > >
> > > >
> > > Just wanted to understand this better. Lets take an example, say we
> have
> > a
> > > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > > Everything is running fine and suddenly C4 host has issues and it
> goes
> > > > down. Now we notice that we can still operate with c1, c2, c3 and
> don't
> > > > want to wait for
> > > > c4 to come back up. We use the admin api
> > > > "invokeConsumerRebalance("GroupA")".
> > > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > > > group again (in there heartBeatResponse) as first step of rebalance.
> > > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3
> is
> > > > delayed. At this stage, if we are not "waiting" on any join group
> > > request,
> > > > few things can happen :
> > > >
> > > >    - c4's partitions are distributed only among c1,c2. c3 maintains
> its
> > > >    original assignment. c1, c2 will start processing the newly
> assigned
> > > >    partitions.
> > > >
> > > > OR
> > > >
> > > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2
> start
> > > >    processing the newly assigned partitions. c3 gets to know about
> the
> > > newly
> > > >    assigned partitions later when it sends the JoinGroupRequest
> (which
> > > was
> > > >    delayed).
> > > >
> > > > OR
> > > >
> > > >    - Will the rebalance do a complete reassignment, where c1, c2, c3
> > have
> > > >    to give up there partitions and all the partitions belonging to
> c1,
> > > c2, c3,
> > > >    c4 will be redistributed among c1, c2, c3 ? If this is the case,
> the
> > > >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> > > revoke
> > > >    there partitions and rejoin the group.
> > > >
> > > > This is as per my understanding of how the KIP would work without
> > > changing
> > > > the underlying group coordination workflow. Please correct me if I
> > > > misunderstood something here.
> > > >
> > >
> > >
> > > - When we say that we would use invokeConsumerRebalance(groupId) to
> down
> > > scale, with the example in the above question, how will the
> > > GroupCoordinator know that c4 should be kicked out of the group since
> we
> > > are trying to invoke rebalance proactively without waiting for c4's
> > session
> > > time out to expire. Should there be a way of telling the
> GroupCoordinator
> > > that consumer c4 has been kicked out of the groupId = "GroupA"?
> > >
> > > - Also it looks like the statement "If the `member.id` uses
> > > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and
> replace
> > > the one within current map, if `group.member.name` is known. Also once
> > we
> > > are done with KIP-394
> > > <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > > >,
> > > all the join group requests are requiring `member.id` to physically
> > enter
> > > the consumer group. This way the latest joined " is incomplete. Can you
> > > take a look at this?
> > > Also when we say "all the join group requests are requiring `member.id
> `
> > to
> > > physically enter the consumer group." because a newly started consumer
> > will
> > > not have a "member.id", I assume you mean, once the GroupCoordinator
> > > assigns a member.id to the newly started consumer, it has to use it
> for
> > > any
> > > future JoinGroupRequests. Is my understanding correct?
> > >
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> > both
> > > > in this reply.
> > > >
> > > >
> > > > >    1. Do you intend to have member.id is a static config like
> > > > member.name
> > > > >    after KIP-345 and KIP-394?
> > > >
> > > > No, we shall only rely on broker to allocate member.id for the
> > consumer
> > > > instances. FYI, I already
> > > >
> > > > started the discussion thread for KIP-394 😊
> > > >
> > > > >    2. Regarding "On client side, we add a new config called
> > MEMBER_NAME
> > > > in
> > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > config
> > > is
> > > > > set,
> > > > >    we will put it in the initial join group request to identify
> > itself
> > > > as a
> > > > >    static member (static membership); otherwise, we will still send
> > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > (dynamic
> > > > >    membership)."
> > > > >       - What is the value of member_id sent in the first
> > > JoinGroupRequest
> > > > >       when member_name is set (using static rebalance)? Is it
> > > > > UNKNOW_MEMBER_ID?
> > > >
> > > > Yes, we could only use unknown member id. Actually this part of the
> > > > proposal is outdated,
> > > >
> > > > let me do another audit of the whole doc. Basically, it is currently
> > > > impossible to send `member.id`
> > > >
> > > > when consumer restarted. Sorry for the confusions!
> > > >
> > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > match
> > > > the
> > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > edge
> > > > case
> > > > >    that if we could have members with the same `member.name` (for
> > > > example
> > > > >    mis-configured instances with a valid member.id but added a
> used
> > > > member
> > > > >    name on runtime). When member name has duplicates, we could
> refuse
> > > > join
> > > > >    request from members with an outdated `member.id` (since we
> > update
> > > > the
> > > > >    mapping upon each join group request). In an edge case where the
> > > > client
> > > > >    hits this exception in the response, it is suggesting that some
> > > other
> > > > >    consumer takes its spot."
> > > > >       - The part of "some other consumer takes the spot" would be
> > > > >       intentional, right? Also when you say " The edge case that if
> > we
> > > > >       could have members with the same `member.name` (for example
> > > > >       mis-configured instances *with a valid member.id <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > >
> > > > > *but
> > > > >       added a used member name on runtime).", what do you mean by
> > > *valid
> > > > >       member id* here? Does it mean that there exist a mapping of
> > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > member.name
> > > > >       <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > >
> > > > = MemberB and member.id <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > >
> > > > =
> > > > > id1 *
> > > > >       ?
> > > >
> > > > I would take Jason's advice that each time we have unknown member
> > joining
> > > > the group, the broker will
> > > >
> > > > always assign a new and unique id to track its identity. In this way,
> > > > consumer with duplicate member name
> > > >
> > > > will be fenced.
> > > >
> > > > >    4. Depending on your explanation for point 2 and the point 3
> above
> > > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > > >    member_name but unknown member_id, if the consumer sends
> > > > > "UNKNOW_MEMBER_ID"
> > > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> > to
> > > > > give it
> > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > >
> > > > Like explained above, we shall not materialize the member.id.
> Instead
> > we
> > > > need to rely on broker to allocate
> > > >
> > > > a unique id for consumer just like what we have now.
> > > >
> > > > >    5. Regarding "So in summary, *the member will only be removed
> due
> > to
> > > > >    session timeout*. We shall remove it from both in-memory static
> > > member
> > > > >    name mapping and member list."
> > > > >       - If the rebalance is invoked manually using the the admin
> > apis,
> > > > how
> > > > >       long should the group coordinator wait for the members of the
> > > > > group to send
> > > > >       a JoinGroupRequest for participating in the rebalance? How
> is a
> > > > > lagging
> > > > >       consumer handled?
> > > >
> > > > The plan is to disable member kick out when rebalance.timeout is
> > reached,
> > > > so basically we are not "waiting" any
> > > >
> > > > join group request from existing members; we shall just rebalance
> base
> > on
> > > > what we currently have within the group
> > > >
> > > > metadata. Lagging consumer will trigger rebalance later if session
> > > timeout
> > > > > rebalance timeout.
> > > >
> > > > >    6. Another detail to take care is that we need to automatically
> > take
> > > > the
> > > > >    hash of group id so that we know which broker to send this
> request
> > > to.
> > > > >       - I assume this should be same as the way we find the
> > > coordinator,
> > > > >       today right? If yes, should we specify it in the KIP ?
> > > >
> > > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > > >
> > > > >    7. Are there any specific failure scenarios when you say "other
> > > > >    potential failure cases."? It would be good to mention them
> > > > explicitly,
> > > > > if
> > > > >    you think there are any.
> > > >
> > > > Nah, I'm gonna remove it because it seems causing more confusion than
> > > > making my assumption clear, which is
> > > >
> > > > "there could be other failure cases that I can't enumerate now" 😊
> > > >
> > > > >    8. It would be good to have a rollback plan as you have for roll
> > > > forward
> > > > >    in the KIP.
> > > >
> > > > Great suggestion! Added a simple rollback plan.
> > > >
> > > >
> > > > Next is answering Jason's suggestions:
> > > >
> > > > 1. This may be the same thing that Mayuresh is asking about. I think
> > the
> > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > member
> > > > name, but no member id, then we will return the current member id
> > > > associated with that name. It seems in this case that we wouldn't be
> > able
> > > > to protect from having two consumers active with the same configured
> > > > member.name? For example, imagine that we had a consumer with
> > > member.name
> > > > =A
> > > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > > instance starts up with member.name=A. If it is also assigned
> > member.id
> > > =1,
> > > > then how can we detect the zombie if it comes back to life? Both
> > > instances
> > > > will have the same member.id.
> > > >
> > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > need
> > > to
> > > > fence previous members. I am wondering if we can generate a new
> > > member.id
> > > > every time we receive a request from a static member with an unknown
> > > member
> > > > id. If the old instance with the same member.name attempts any
> > > operation,
> > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> the
> > > > subscription of the new instance hasn't changed, then we can skip the
> > > > rebalance and return the current assignment without forcing a
> > rebalance.
> > > >
> > > > The trick to making this work is in the error handling of the zombie
> > > > consumer. If the zombie simply resets its member.id and rejoins to
> > get a
> > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> > up
> > > > fencing the new member. We want to avoid this. There needs to be an
> > > > expectation for static members that the member.id of a static member
> > > will
> > > > not be changed except when a new member with the same member.name
> > joins
> > > > the
> > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > consumers
> > > > with static member names.
> > > >
> > > > Yep, I like this idea! Keep giving out refresh member.id when facing
> > > > anonymous request will definitely
> > > >
> > > > prevent processing bug due to duplicate consumers, however I don't
> > think
> > > I
> > > > fully understand the 3rd paragraph where
> > > >
> > > > you mentioned  "There needs to be an expectation for static members
> > that
> > > > the member.id of a static member will
> > > >
> > > > not be changed except when a new member with the same member.name
> > joins
> > > > the group. "  How do you plan
> > > > to know whether this member is new member or old member? I feel even
> > with
> > > > zombie consumer takes the ownership,
> > > > it should be detected very quickly (as MISMATCH_ID exception trigger
> > > > original consumer instance dies)
> > > > and end user will start to fix it right away. Is there any similar
> > logic
> > > > we applied in fencing duplicate `transaction.id`?
> > > >
> > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> > far
> > > as
> > > > I understand it, it is used for scaling down a consumer group and
> > somehow
> > > > bypasses normal session timeout expiration. I am wondering how
> critical
> > > > this piece is and whether we can leave it for future work. If not,
> then
> > > it
> > > > would be helpful to elaborate on its implementation. How would the
> > > > coordinator know which members to kick out of the group?
> > > >
> > > > This API is needed when we need to immediately trigger rebalance
> > instead
> > > > of waiting session timeout
> > > >
> > > > or rebalance timeout (Emergent scale up/down). It is very necessary
> to
> > > > have it for
> > > >
> > > > management purpose because user could choose when to trigger
> rebalance
> > > > pretty freely,
> > > >
> > > > gaining more client side control.
> > > >
> > > > In the meanwhile I see your point that we need to actually have the
> > > > ability to kick out members that we plan
> > > >
> > > > to scale down fast (as rebalance timeout no longer kicks any offline
> > > > member out of the group), I will think of adding an optional
> > > >
> > > > list of members that are ready to be removed.
> > > >
> > > > Another idea is to let static member send `LeaveGroupRequest` when
> they
> > > > are going offline (either scale down or bouncing),
> > > >
> > > > and broker will cache this information as "OfflineMembers" without
> > > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> > will
> > > >
> > > > kick the static members that are currently offline and trigger
> > rebalance
> > > > immediately. How does this plan sound?
> > > >
> > > > 3. I've been holding back on mentioning this, but I think we should
> > > > reconsider the name `member.name`. I think we want something that
> > > suggests
> > > > its expectation of uniqueness in the group. How about `
> > group.instance.id
> > > `
> > > > to go along with `group.id`?
> > > >
> > > > Yea, Dong and Stanislav also mentioned this naming. I personally buy
> in
> > > > the namespace idea, and
> > > >
> > > > since we already use `member.name` in a lot of context, I decide to
> > > > rename the config to `group.member.name`
> > > >
> > > > which should be sufficient for solving all the concerns we have now.
> > > > Sounds good?
> > > >
> > > >
> > > > Thank you for your great suggestions! Let me know if my reply makes
> > sense
> > > > her.
> > > >
> > > >
> > > > Best,
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Jason Gustafson <ja...@confluent.io>
> > > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > > To: dev
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the updates. Looks like we're headed in the right
> direction
> > > and
> > > > clearly the interest that this KIP is receiving shows how strong the
> > > > motivation is!
> > > >
> > > > I have a few questions:
> > > >
> > > > 1. This may be the same thing that Mayuresh is asking about. I think
> > the
> > > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> > member
> > > > name, but no member id, then we will return the current member id
> > > > associated with that name. It seems in this case that we wouldn't be
> > able
> > > > to protect from having two consumers active with the same configured
> > > > member.name? For example, imagine that we had a consumer with
> > > member.name
> > > > =A
> > > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > > instance starts up with member.name=A. If it is also assigned
> > member.id
> > > =1,
> > > > then how can we detect the zombie if it comes back to life? Both
> > > instances
> > > > will have the same member.id.
> > > >
> > > > The goal is to avoid a rebalance on a rolling restart, but we still
> > need
> > > to
> > > > fence previous members. I am wondering if we can generate a new
> > > member.id
> > > > every time we receive a request from a static member with an unknown
> > > member
> > > > id. If the old instance with the same member.name attempts any
> > > operation,
> > > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as
> the
> > > > subscription of the new instance hasn't changed, then we can skip the
> > > > rebalance and return the current assignment without forcing a
> > rebalance.
> > > >
> > > > The trick to making this work is in the error handling of the zombie
> > > > consumer. If the zombie simply resets its member.id and rejoins to
> > get a
> > > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> > up
> > > > fencing the new member. We want to avoid this. There needs to be an
> > > > expectation for static members that the member.id of a static member
> > > will
> > > > not be changed except when a new member with the same member.name
> > joins
> > > > the
> > > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> > consumers
> > > > with static member names.
> > > >
> > > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> > far
> > > as
> > > > I understand it, it is used for scaling down a consumer group and
> > somehow
> > > > bypasses normal session timeout expiration. I am wondering how
> critical
> > > > this piece is and whether we can leave it for future work. If not,
> then
> > > it
> > > > would be helpful to elaborate on its implementation. How would the
> > > > coordinator know which members to kick out of the group?
> > > >
> > > > 3. I've been holding back on mentioning this, but I think we should
> > > > reconsider the name `member.name`. I think we want something that
> > > suggests
> > > > its expectation of uniqueness in the group. How about `
> > group.instance.id
> > > `
> > > > to go along with `group.id`?
> > > >
> > > > Thanks,
> > > > Jason
> > > >
> > > >
> > > >
> > > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > > gharatmayuresh15@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks a lot for replying to all the queries and discussions here,
> so
> > > > > patiently.
> > > > > Really appreciate it.
> > > > >
> > > > > Had a few questions and suggestions after rereading the current
> > version
> > > > of
> > > > > the KIP :
> > > > >
> > > > >
> > > > >    1. Do you intend to have member.id is a static config like
> > > > member.name
> > > > >    after KIP-345 and KIP-394?
> > > > >    2. Regarding "On client side, we add a new config called
> > MEMBER_NAME
> > > > in
> > > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> > config
> > > is
> > > > > set,
> > > > >    we will put it in the initial join group request to identify
> > itself
> > > > as a
> > > > >    static member (static membership); otherwise, we will still send
> > > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > > (dynamic
> > > > >    membership)."
> > > > >       - What is the value of member_id sent in the first
> > > JoinGroupRequest
> > > > >       when member_name is set (using static rebalance)? Is it
> > > > > UNKNOW_MEMBER_ID?
> > > > >    3. Regarding "we are requiring member.id (if not unknown) to
> > match
> > > > the
> > > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> > edge
> > > > case
> > > > >    that if we could have members with the same `member.name` (for
> > > > example
> > > > >    mis-configured instances with a valid member.id but added a
> used
> > > > member
> > > > >    name on runtime). When member name has duplicates, we could
> refuse
> > > > join
> > > > >    request from members with an outdated `member.id` (since we
> > update
> > > > the
> > > > >    mapping upon each join group request). In an edge case where the
> > > > client
> > > > >    hits this exception in the response, it is suggesting that some
> > > other
> > > > >    consumer takes its spot."
> > > > >       - The part of "some other consumer takes the spot" would be
> > > > >       intentional, right? Also when you say " The edge case that if
> > we
> > > > >       could have members with the same `member.name` (for example
> > > > >       mis-configured instances *with a valid member.id <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > > >
> > > > > *but
> > > > >       added a used member name on runtime).", what do you mean by
> > > *valid
> > > > >       member id* here? Does it mean that there exist a mapping of
> > > > >       member.name to member.id like *MemberA -> id1* on the
> > > > >       GroupCoordinator and this consumer is trying to join with *
> > > > > member.name
> > > > >       <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > >
> > > > = MemberB and member.id <
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > >
> > > > =
> > > > > id1 *
> > > > >       ?
> > > > >    4. Depending on your explanation for point 2 and the point 3
> above
> > > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > > >    member_name but unknown member_id, if the consumer sends
> > > > > "UNKNOW_MEMBER_ID"
> > > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> > to
> > > > > give it
> > > > >    a member_id, is the consumer suppose to remember member_id for
> > > > >    joinGroupRequests? If yes, how are restarts handled?
> > > > >    5. Regarding "So in summary, *the member will only be removed
> due
> > to
> > > > >    session timeout*. We shall remove it from both in-memory static
> > > member
> > > > >    name mapping and member list."
> > > > >       - If the rebalance is invoked manually using the the admin
> > apis,
> > > > how
> > > > >       long should the group coordinator wait for the members of the
> > > > > group to send
> > > > >       a JoinGroupRequest for participating in the rebalance? How
> is a
> > > > > lagging
> > > > >       consumer handled?
> > > > >    6. Another detail to take care is that we need to automatically
> > take
> > > > the
> > > > >    hash of group id so that we know which broker to send this
> request
> > > to.
> > > > >       - I assume this should be same as the way we find the
> > > coordinator,
> > > > >       today right? If yes, should we specify it in the KIP ?
> > > > >    7. Are there any specific failure scenarios when you say "other
> > > > >    potential failure cases."? It would be good to mention them
> > > > explicitly,
> > > > > if
> > > > >    you think there are any.
> > > > >    8. It would be good to have a rollback plan as you have for roll
> > > > forward
> > > > >    in the KIP.
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > > gharatmayuresh15@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi Boyang,
> > > > > >
> > > > > > Do you have a discuss thread for KIP-394 that you mentioned here
> ?
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Mayuresh
> > > > > >
> > > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bchen11@outlook.com
> >
> > > > wrote:
> > > > > >
> > > > > >> Hey Dong, thanks for the follow-up here!
> > > > > >>
> > > > > >>
> > > > > >> 1) It is not very clear to the user what is the difference
> between
> > > > > >> member.name and client.id as both seems to be used to identify
> > the
> > > > > >> consumer. I am wondering if it would be more intuitive to name
> it
> > > > > >> group.member.name (preferred choice since it matches the
> current
> > > > > group.id
> > > > > >> config name) or rebalance.member.name to explicitly show that
> the
> > > id
> > > > is
> > > > > >> solely used for rebalance.
> > > > > >> Great question. I feel `member.name` is enough to explain
> itself,
> > > it
> > > > > >> seems not very
> > > > > >> helpful to make the config name longer. Comparing `name` with
> `id`
> > > > gives
> > > > > >> user the
> > > > > >> impression that they have the control over it with customized
> rule
> > > > than
> > > > > >> library decided.
> > > > > >>
> > > > > >> 2) In the interface change section it is said that
> > > > > >> GroupMaxSessionTimeoutMs
> > > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > > change
> > > > > the
> > > > > >> default value of this config. It does not seem necessary to
> > increase
> > > > the
> > > > > >> time of consumer failure detection when user doesn't use static
> > > > > >> membership.
> > > > > >> Also, say static membership is enabled, then this default config
> > > > change
> > > > > >> will cause a partition to be unavailable for consumption for 30
> > > > minutes
> > > > > if
> > > > > >> there is hard consumer failure, which seems to be worse
> experience
> > > > than
> > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > particularly
> > > > > >> for
> > > > > >> new users of Kafka. Could you explain more why we should make
> this
> > > > > change?
> > > > > >> We are not changing the default session timeout value. We are
> just
> > > > > >> changing the
> > > > > >> cap we are enforcing on the session timeout max value. So this
> > > change
> > > > is
> > > > > >> not affecting
> > > > > >> what kind of membership end user is using, and loosing the cap
> is
> > > > giving
> > > > > >> end user
> > > > > >> more flexibility on trade-off between liveness and stability.
> > > > > >>
> > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > DUPLICATE_STATIC_MEMBER
> > > > > >> into one error? It seems that these two errors are currently
> > handled
> > > > by
> > > > > >> the
> > > > > >> consumer in the same way. And we don't also don't expect
> > > > > >> MEMBER_ID_MISMATCH
> > > > > >> to happen. Thus it is not clear what is the benefit of having
> two
> > > > > errors.
> > > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> > because
> > > > with
> > > > > >> the KIP-394<
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=E3cqYTMRFsAs5TQI4JxHm3kOWCfkVWjpuc%2BuNHezwG0%3D&amp;reserved=0
> > > > > >> >
> > > > > >> we will automatically fence all join requests with
> > > UNKNOWN_MEMBER_ID.
> > > > > >>
> > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > > contains
> > > > > >> member name which is already in the consumer group, however the
> > > member
> > > > > id
> > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> > has
> > > > not
> > > > > >> expired this member from the memory) and memberId
> > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > persisted
> > > > > >> across consumer restart in the consumer side). Does it mean that
> > > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > > rejected
> > > > > >> until the sessionTimeoutMs has passed?
> > > > > >> Same answer as question 3). This part of the logic shall be
> > removed
> > > > from
> > > > > >> the proposal.
> > > > > >>
> > > > > >> 5) It seems that we always add two methods to the interface
> > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> options
> > > and
> > > > > the
> > > > > >> other without option. Could this be specified in the interface
> > > change
> > > > > >> section?
> > > > > >> Sounds good! Added both methods.
> > > > > >>
> > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > > trigger
> > > > > >> rebalance? If so, we probably want to specify the command line
> > tool
> > > > > >> interface similar to
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > > >> .
> > > > > >> Added the script.
> > > > > >>
> > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> with
> > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> extra
> > > > > meaning
> > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > seems
> > > > > >> simpler
> > > > > >> to allows this API to trigger rebalance regardless of whether
> > > consumer
> > > > > is
> > > > > >> configured with memberName.
> > > > > >> Sounds good. Right now I feel for both static and dynamic
> > membership
> > > > it
> > > > > is
> > > > > >> more manageable to introduce the consumer rebalance method
> through
> > > > admin
> > > > > >> client API.
> > > > > >>
> > > > > >> 8) It is not very clear how the newly added AdminClient API
> > trigger
> > > > > >> rebalance. For example, does it send request? Can this be
> > explained
> > > in
> > > > > the
> > > > > >> KIP?
> > > > > >>
> > > > > >> Sure, I will add more details to the API.
> > > > > >>
> > > > > >>
> > > > > >> Thanks again for the helpful suggestions!
> > > > > >>
> > > > > >>
> > > > > >> Best,
> > > > > >> Boyang
> > > > > >>
> > > > > >> ________________________________
> > > > > >> From: Dong Lin <li...@gmail.com>
> > > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > > >> To: dev
> > > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > > >> specifying member id
> > > > > >>
> > > > > >> Hey Boyang,
> > > > > >>
> > > > > >> Thanks for the update! Here are some followup comments:
> > > > > >>
> > > > > >> 1) It is not very clear to the user what is the difference
> between
> > > > > >> member.name and client.id as both seems to be used to identify
> > the
> > > > > >> consumer. I am wondering if it would be more intuitive to name
> it
> > > > > >> group.member.name (preferred choice since it matches the
> current
> > > > > group.id
> > > > > >> config name) or rebalance.member.name to explicitly show that
> the
> > > id
> > > > is
> > > > > >> solely used for rebalance.
> > > > > >>
> > > > > >> 2) In the interface change section it is said that
> > > > > >> GroupMaxSessionTimeoutMs
> > > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > > change
> > > > > the
> > > > > >> default value of this config. It does not seem necessary to
> > increase
> > > > the
> > > > > >> time of consumer failure detection when user doesn't use static
> > > > > >> membership.
> > > > > >> Also, say static membership is enabled, then this default config
> > > > change
> > > > > >> will cause a partition to be unavailable for consumption for 30
> > > > minutes
> > > > > if
> > > > > >> there is hard consumer failure, which seems to be worse
> experience
> > > > than
> > > > > >> having unnecessary rebalance (when this timeout is small),
> > > > particularly
> > > > > >> for
> > > > > >> new users of Kafka. Could you explain more why we should make
> this
> > > > > change?
> > > > > >>
> > > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > > DUPLICATE_STATIC_MEMBER
> > > > > >> into one error? It seems that these two errors are currently
> > handled
> > > > by
> > > > > >> the
> > > > > >> consumer in the same way. And we don't also don't expect
> > > > > >> MEMBER_ID_MISMATCH
> > > > > >> to happen. Thus it is not clear what is the benefit of having
> two
> > > > > errors.
> > > > > >>
> > > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > > contains
> > > > > >> member name which is already in the consumer group, however the
> > > member
> > > > > id
> > > > > >> was missing". After a consumer is restarted, it will send a
> > > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> > has
> > > > not
> > > > > >> expired this member from the memory) and memberId
> > > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > > persisted
> > > > > >> across consumer restart in the consumer side). Does it mean that
> > > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > > rejected
> > > > > >> until the sessionTimeoutMs has passed?
> > > > > >>
> > > > > >> 5) It seems that we always add two methods to the interface
> > > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with
> options
> > > and
> > > > > the
> > > > > >> other without option. Could this be specified in the interface
> > > change
> > > > > >> section?
> > > > > >>
> > > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > > trigger
> > > > > >> rebalance? If so, we probably want to specify the command line
> > tool
> > > > > >> interface similar to
> > > > > >>
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > > >> .
> > > > > >>
> > > > > >> 7) Would it be simpler to replace name "forceStaticRebalance"
> with
> > > > > >> "invokeConsumerRebalance"? It is not very clear what is the
> extra
> > > > > meaning
> > > > > >> of world "force" as compared to "trigger" or "invoke". And it
> > seems
> > > > > >> simpler
> > > > > >> to allows this API to trigger rebalance regardless of whether
> > > consumer
> > > > > is
> > > > > >> configured with memberName.
> > > > > >>
> > > > > >> 8) It is not very clear how the newly added AdminClient API
> > trigger
> > > > > >> rebalance. For example, does it send request? Can this be
> > explained
> > > in
> > > > > the
> > > > > >> KIP?
> > > > > >>
> > > > > >> Thanks,
> > > > > >> Dong
> > > > > >>
> > > > > >>
> > > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <
> bchen11@outlook.com>
> > > > > wrote:
> > > > > >>
> > > > > >> > Hey Mayuresh,
> > > > > >> >
> > > > > >> >
> > > > > >> > thanks for your feedbacks! I will try do another checklist
> here.
> > > > > >> >
> > > > > >> >
> > > > > >> > > By this you mean, even if the application has not called
> > > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > > sending
> > > > > >> the
> > > > > >> > > LeaveGroup request, right?
> > > > > >> >
> > > > > >> > Yep it's true, we will prevent client from sending leave group
> > > > request
> > > > > >> > when they are set with `member.name`.
> > > > > >> >
> > > > > >> >
> > > > > >> > > When is the member.name removed from this map?
> > > > > >> > Good question, we will only kick off member due to session
> > timeout
> > > > > >> within
> > > > > >> > static membership. Let me update the KIP to clearly assert
> that.
> > > > > >> >
> > > > > >> > > How is this case (missing member id) handled on the client
> > side?
> > > > > What
> > > > > >> is
> > > > > >> > the application that
> > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > >> > I have extended the two exceptions within join group response
> > V4.
> > > > > >> > Basically I define both corresponding actions to be immediate
> > > > failing
> > > > > >> > client application, because so far it is unknown what kind of
> > > client
> > > > > >> issue
> > > > > >> > could trigger them. After the first version, we will keep
> > enhance
> > > > the
> > > > > >> error
> > > > > >> > handling logic!
> > > > > >> >
> > > > > >> > > This would mean that it might take more time to detect
> unowned
> > > > topic
> > > > > >> > > partitions and may cause delay for applications that perform
> > > data
> > > > > >> > mirroring
> > > > > >> > > tasks. I discussed this with our sre and we have a
> suggestion
> > to
> > > > > make
> > > > > >> > here
> > > > > >> > > as listed below separately.
> > > > > >> > The goal of extending session timeout cap is for users with
> good
> > > > > client
> > > > > >> > side monitoring tools that could auto-heal the dead consumers
> > very
> > > > > >> fast. So
> > > > > >> > it is optional (and personal) to extend session timeout to a
> > > > > reasonable
> > > > > >> > number with different client scenarios.
> > > > > >> >
> > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > > >> >
> > > > > >> > > What do you mean by " Internally we would optimize this
> logic
> > by
> > > > > >> having
> > > > > >> > > rebalance timeout only in charge of stopping prepare
> rebalance
> > > > > stage,
> > > > > >> > > without removing non-responsive members immediately." There
> > > would
> > > > > not
> > > > > >> be
> > > > > >> > a
> > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > request
> > > > > later,
> > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > > functionality
> > > > > >> > to only use as a timer to
> > > > > >> > end prepare rebalance stage. This way, late joining static
> > members
> > > > > will
> > > > > >> > not trigger further rebalance
> > > > > >> > as long as they are within session timeout. I added your
> > highlight
> > > > to
> > > > > >> the
> > > > > >> > KIP!
> > > > > >> >
> > > > > >> > > The KIP talks about scale up scenario but its not quite
> clear
> > > how
> > > > we
> > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> we
> > > > adding
> > > > > >> > status
> > > > > >> > > "learner" ?. Can you shed more light on how this is handled
> in
> > > the
> > > > > >> KIP,
> > > > > >> > if
> > > > > >> > > its handled?
> > > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> > because
> > > we
> > > > > >> > believe client side could
> > > > > >> > better handle this logic.
> > > > > >> >
> > > > > >> > > I think Jason had brought this up earlier about having a way
> > to
> > > > say
> > > > > >> how
> > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > consumer
> > > > > >> group.
> > > > > >> > > If we can do this, then in case of mirroring applications we
> > can
> > > > do
> > > > > >> this
> > > > > >> > :
> > > > > >> > > Lets say we have a mirroring application that consumes from
> > > Kafka
> > > > > >> cluster
> > > > > >> > > A and produces to Kafka cluster B.
> > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > Kafka
> > > > > >> service
> > > > > >> > > providers can set a mirroring group saying that it will
> take,
> > > for
> > > > > >> example
> > > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> > and
> > > > > >> latency
> > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > spare
> > > > in
> > > > > >> the
> > > > > >> > > same group.
> > > > > >> > > So when the first 300 members/consumers to join the group
> will
> > > > start
> > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > >> > > The moment one of the consumer (for example: consumer number
> > 54)
> > > > > from
> > > > > >> the
> > > > > >> > > first 300 members go out of the group (crossed session
> > timeout),
> > > > it
> > > > > >> (the
> > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> the
> > > > > >> consumer
> > > > > >> > > member 54 to one of the spare hosts.
> > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > being a
> > > > > >> part
> > > > > >> > of
> > > > > >> > > the spare pool.
> > > > > >> > > This enables us to have lower session timeouts and low
> latency
> > > > > >> mirroring,
> > > > > >> > > in cases where the service providers are OK with having
> spare
> > > > hosts.
> > > > > >> > > This would mean that we would tolerate n consumer members
> > > leaving
> > > > > and
> > > > > >> > > rejoining the group and still provide low latency as long
> as n
> > > <=
> > > > > >> number
> > > > > >> > of
> > > > > >> > > spare consumers.
> > > > > >> > > If there are no spare host available, we can get back to the
> > > idea
> > > > as
> > > > > >> > > described in the KIP.
> > > > > >> > Great idea! In fact on top of static membership we could later
> > > > > introduce
> > > > > >> > APIs to set hard-coded
> > > > > >> > client ids to the group and replace the dead host, or as you
> > > > proposed
> > > > > to
> > > > > >> > define spare host as
> > > > > >> > what I understood as hot backup. I will put both Jason and
> your
> > > > > >> > suggestions into a separate section
> > > > > >> > called "Future works". Note that this spare host idea may be
> > also
> > > > > >> solvable
> > > > > >> > through rebalance protocol
> > > > > >> > IMO.
> > > > > >> >
> > > > > >> > Thank you again for the great feedback!
> > > > > >> >
> > > > > >> > Boyang
> > > > > >> > ________________________________
> > > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > > >> > To: dev@kafka.apache.org
> > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > by
> > > > > >> > specifying member id
> > > > > >> >
> > > > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> > > email
> > > > > on
> > > > > >> my
> > > > > >> > thread, so I will just do a checklist here!
> > > > > >> >
> > > > > >> >
> > > > > >> > 1) The motivation currently explicitly states that the goal is
> > to
> > > > > >> improve
> > > > > >> >
> > > > > >> > performance for heavy state application. It seems that the
> > > > motivation
> > > > > >> can
> > > > > >> >
> > > > > >> > be stronger with the following use-case. Currently for
> > MirrorMaker
> > > > > >> cluster
> > > > > >> >
> > > > > >> > with e.g. 100 MirrorMaker processes, it will take a long time
> to
> > > > > rolling
> > > > > >> >
> > > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker
> process
> > > > > restart
> > > > > >> >
> > > > > >> > will trigger a rebalance which currently pause the consumption
> > of
> > > > the
> > > > > >> all
> > > > > >> >
> > > > > >> > partitions of the MirrorMaker cluster. With the change stated
> in
> > > > this
> > > > > >> >
> > > > > >> > patch, as long as a MirrorMaker can restart within the
> specified
> > > > > timeout
> > > > > >> >
> > > > > >> > (e.g. 2 minutes), then we only need constant number of
> rebalance
> > > > (e.g.
> > > > > >> for
> > > > > >> >
> > > > > >> > leader restart) for the entire rolling bounce, which will
> > > > > significantly
> > > > > >> >
> > > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > > opinion,
> > > > > >> the
> > > > > >> >
> > > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if
> the
> > > > > >> consumer
> > > > > >> >
> > > > > >> > process can be restarted within soon, which helps performance
> > even
> > > > if
> > > > > >> >
> > > > > >> > overhead of state shuffling for a given process is small.
> > > > > >> >
> > > > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > > > making
> > > > > >> the
> > > > > >> > motivation more solid!
> > > > > >> >
> > > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > > writeup
> > > > > >> style
> > > > > >> > of other KIP (e.g. KIP-98) and list the interface change such
> as
> > > new
> > > > > >> > configs (e.g. registration timeout), new request/response, new
> > > > > >> AdminClient
> > > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> Currently
> > > > some
> > > > > of
> > > > > >> > these are specified in the Proposed Change section which makes
> > it
> > > a
> > > > > bit
> > > > > >> > inconvenient to understand the new interface that will be
> > exposed
> > > to
> > > > > >> user.
> > > > > >> > Explanation of the current two-phase rebalance protocol
> probably
> > > can
> > > > > be
> > > > > >> > moved out of public interface section.
> > > > > >> > This is a great suggestion! I just consolidated all the public
> > API
> > > > > >> > changes, and the whole KIP
> > > > > >> > looks much more organized!
> > > > > >> >
> > > > > >> > 3) There are currently two version of JoinGroupRequest in the
> > KIP
> > > > and
> > > > > >> only
> > > > > >> > one of them has field memberId. This seems confusing.
> > > > > >> > Yep, I already found this issue and fixed it.
> > > > > >> >
> > > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > > rebalance
> > > > > >> could
> > > > > >> > be helpful here, but we will make a call once we finished the
> > > major
> > > > > >> > implementation". So this seems to be still an open question in
> > the
> > > > > >> current
> > > > > >> > design. We probably want to agree on this before voting for
> the
> > > KIP.
> > > > > >> > We have finalized the idea that this API is needed.
> > > > > >> >
> > > > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> > Can
> > > > you
> > > > > >> > specify the name of the config key and the default config
> value?
> > > > > >> Possible
> > > > > >> > default values include empty string or null (similar to
> > > > > transaction.id<
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=qWkxpqJMF6ugtVKhupIthbcmSNFmp4sX5EfgvKAiAQo%3D&amp;reserved=0
> > > > > >> >
> > > > > >> > in
> > > > > >> > producer config).
> > > > > >> > I have defined the `member.name` in "New configuration"
> > section.
> > > > > >> >
> > > > > >> > 6) Regarding the use of the topic "static_member_map" to
> persist
> > > > > member
> > > > > >> > name map, currently if consumer coordinator broker goes
> offline,
> > > > > >> rebalance
> > > > > >> > is triggered and consumers will try connect to the new
> > > coordinator.
> > > > If
> > > > > >> > these consumers can connect to the new coordinator within
> > > > > >> > max.poll.interval.ms<
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > > >> >
> > > > > >> > which by default is 5 minutes, given that broker can
> > > > > >> > use a deterministic algorithm to determine the partition ->
> > > > > member_name
> > > > > >> > mapping, each consumer should get assigned the same set of
> > > > partitions
> > > > > >> > without requiring state shuffling. So it is not clear whether
> we
> > > > have
> > > > > a
> > > > > >> > strong use-case for this new logic. Can you help clarify what
> is
> > > the
> > > > > >> > benefit of using topic "static_member_map" to persist member
> > name
> > > > map?
> > > > > >> > I have discussed with Guozhang offline, and I believe reusing
> > the
> > > > > >> current
> > > > > >> > `_consumer_offsets`
> > > > > >> > topic is a better and unified solution.
> > > > > >> >
> > > > > >> > 7) Regarding the introduction of the expensionTimeoutMs
> config,
> > it
> > > > is
> > > > > >> > mentioned that "we are using expansion timeout to replace
> > > rebalance
> > > > > >> > timeout, which is configured by max.poll.intervals from client
> > > side,
> > > > > and
> > > > > >> > using registration timeout to replace session timeout".
> > Currently
> > > > the
> > > > > >> > default max.poll.interval.ms<
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > > >> >
> > > > > >> > is configured to be 5 minutes and there will
> > > > > >> > be only one rebalance if all new consumers can join within 5
> > > > minutes.
> > > > > >> So it
> > > > > >> > is not clear whether we have a strong use-case for this new
> > > config.
> > > > > Can
> > > > > >> you
> > > > > >> > explain what is the benefit of introducing this new config?
> > > > > >> > Previously our goal is to use expansion timeout as a
> workaround
> > > for
> > > > > >> > triggering multiple
> > > > > >> > rebalances when scaling up members are not joining at the same
> > > time.
> > > > > It
> > > > > >> is
> > > > > >> > decided to
> > > > > >> > be addressed by client side protocol change, so we will not
> > > > introduce
> > > > > >> > expansion timeout.
> > > > > >> >
> > > > > >> > 8) It is mentioned that "To distinguish between previous
> version
> > > of
> > > > > >> > protocol, we will also increase the join group request version
> > to
> > > v4
> > > > > >> when
> > > > > >> > MEMBER_NAME is set" and "If the broker version is not the
> latest
> > > (<
> > > > > v4),
> > > > > >> > the join group request shall be downgraded to v3 without
> setting
> > > the
> > > > > >> member
> > > > > >> > Id". It is probably simpler to just say that this feature is
> > > enabled
> > > > > if
> > > > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > > > >> MEMBER_NAME
> > > > > >> > is configured with non-empty string.
> > > > > >> > Yep, addressed this!
> > > > > >> >
> > > > > >> > 9) It is mentioned that broker may return
> > > NO_STATIC_MEMBER_INFO_SET
> > > > > >> error
> > > > > >> > in OffsetCommitResponse for "commit requests under static
> > > > membership".
> > > > > >> Can
> > > > > >> > you clarify how broker determines whether the commit request
> is
> > > > under
> > > > > >> > static membership?
> > > > > >> >
> > > > > >> > We have agreed that commit request shouldn't be affected by
> the
> > > new
> > > > > >> > membership, thus
> > > > > >> > removing it here. Thanks for catching this!
> > > > > >> >
> > > > > >> > Let me know if you have further suggestions or concerns. Thank
> > you
> > > > for
> > > > > >> > your valuable feedback
> > > > > >> > to help me design the KIP better! (And I will try to address
> > your
> > > > > >> > feedbacks in next round Mayuresh ??)
> > > > > >> >
> > > > > >> > Best,
> > > > > >> > Boyang
> > > > > >> > ________________________________
> > > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > > >> > To: dev@kafka.apache.org
> > > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > by
> > > > > >> > specifying member id
> > > > > >> >
> > > > > >> > Hi Boyang,
> > > > > >> >
> > > > > >> > Thanks for updating the KIP. This is a step good direction for
> > > > > stateful
> > > > > >> > applications and also mirroring applications whose latency is
> > > > affected
> > > > > >> due
> > > > > >> > to the rebalance issues that we have today.
> > > > > >> >
> > > > > >> > I had a few questions on the current version of the KIP :
> > > > > >> > For the effectiveness of the KIP, consumer with member.name
> set
> > > > will
> > > > > >> *not
> > > > > >> > send leave group request* when they go offline
> > > > > >> >
> > > > > >> > > By this you mean, even if the application has not called
> > > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > > sending
> > > > > >> the
> > > > > >> > > LeaveGroup request, right?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > > member.id
> > > > > }
> > > > > >> to
> > > > > >> > track member uniqueness.
> > > > > >> >
> > > > > >> > > When is the member.name removed from this map?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Member.id must be set if the *member.name <
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > > > >> >
> > > > > >> > *is already
> > > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > > >> >
> > > > > >> > > How is this case handled on the client side? What is the
> > > > application
> > > > > >> that
> > > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Session timeout is the timeout we will trigger rebalance when
> a
> > > > member
> > > > > >> goes
> > > > > >> > offline for too long (not sending heartbeat request). To make
> > > static
> > > > > >> > membership effective, we should increase the default max
> session
> > > > > >> timeout to
> > > > > >> > 30 min so that end user could config it freely.
> > > > > >> >
> > > > > >> > > This would mean that it might take more time to detect
> unowned
> > > > topic
> > > > > >> > > partitions and may cause delay for applications that perform
> > > data
> > > > > >> > mirroring
> > > > > >> > > tasks. I discussed this with our sre and we have a
> suggestion
> > to
> > > > > make
> > > > > >> > here
> > > > > >> > > as listed below separately.
> > > > > >> > >
> > > > > >> >
> > > > > >> > Currently there is a config called *rebalance timeout* which
> is
> > > > > >> configured
> > > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > > interval
> > > > > >> is
> > > > > >> > because consumer could only send request within the call of
> > poll()
> > > > and
> > > > > >> we
> > > > > >> > want to wait sufficient time for the join group request. When
> > > > reaching
> > > > > >> > rebalance timeout, the group will move towards
> > completingRebalance
> > > > > stage
> > > > > >> > and remove unjoined groups
> > > > > >> >
> > > > > >> > > you meant remove unjoined members of the group, right ?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Currently there is a config called *rebalance timeout* which
> is
> > > > > >> configured
> > > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > > interval
> > > > > >> is
> > > > > >> > because consumer could only send request within the call of
> > poll()
> > > > and
> > > > > >> we
> > > > > >> > want to wait sufficient time for the join group request. When
> > > > reaching
> > > > > >> > rebalance timeout, the group will move towards
> > completingRebalance
> > > > > stage
> > > > > >> > and remove unjoined groups. This is actually conflicting with
> > the
> > > > > >> design of
> > > > > >> > static membership, because those temporarily unavailable
> members
> > > > will
> > > > > >> > potentially reattempt the join group and trigger extra
> > rebalances.
> > > > > >> > Internally we would optimize this logic by having rebalance
> > > timeout
> > > > > >> only in
> > > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > > >> non-responsive
> > > > > >> > members immediately.
> > > > > >> >
> > > > > >> > > What do you mean by " Internally we would optimize this
> logic
> > by
> > > > > >> having
> > > > > >> > > rebalance timeout only in charge of stopping prepare
> rebalance
> > > > > stage,
> > > > > >> > > without removing non-responsive members immediately." There
> > > would
> > > > > not
> > > > > >> be
> > > > > >> > a
> > > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> > request
> > > > > later,
> > > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > > >> > >
> > > > > >> >
> > > > > >> > Scale Up
> > > > > >> >
> > > > > >> > > The KIP talks about scale up scenario but its not quite
> clear
> > > how
> > > > we
> > > > > >> > > handle it. Are we adding a separate "expansion.timeout" or
> we
> > > > adding
> > > > > >> > status
> > > > > >> > > "learner" ?. Can you shed more light on how this is handled
> in
> > > the
> > > > > >> KIP,
> > > > > >> > if
> > > > > >> > > its handled?
> > > > > >> > >
> > > > > >> >
> > > > > >> >
> > > > > >> > *Discussion*
> > > > > >> > Larger session timeouts causing latency rise for getting data
> > for
> > > > > >> un-owned
> > > > > >> > topic partitions :
> > > > > >> >
> > > > > >> > > I think Jason had brought this up earlier about having a way
> > to
> > > > say
> > > > > >> how
> > > > > >> > > many members/consumer hosts are you choosing to be in the
> > > consumer
> > > > > >> group.
> > > > > >> > > If we can do this, then in case of mirroring applications we
> > can
> > > > do
> > > > > >> this
> > > > > >> > :
> > > > > >> > > Lets say we have a mirroring application that consumes from
> > > Kafka
> > > > > >> cluster
> > > > > >> > > A and produces to Kafka cluster B.
> > > > > >> > > Depending on the data and the Kafka cluster configuration,
> > Kafka
> > > > > >> service
> > > > > >> > > providers can set a mirroring group saying that it will
> take,
> > > for
> > > > > >> example
> > > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> > and
> > > > > >> latency
> > > > > >> > > for mirroring and can have additional 10 consumer hosts as
> > spare
> > > > in
> > > > > >> the
> > > > > >> > > same group.
> > > > > >> > > So when the first 300 members/consumers to join the group
> will
> > > > start
> > > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > > >> > > The remaining 10 consumer members can sit idle.
> > > > > >> > > The moment one of the consumer (for example: consumer number
> > 54)
> > > > > from
> > > > > >> the
> > > > > >> > > first 300 members go out of the group (crossed session
> > timeout),
> > > > it
> > > > > >> (the
> > > > > >> > > groupCoordinator) can just assign the topicPartitions from
> the
> > > > > >> consumer
> > > > > >> > > member 54 to one of the spare hosts.
> > > > > >> > > Once the consumer member 54 comes back up, it can start as
> > > being a
> > > > > >> part
> > > > > >> > of
> > > > > >> > > the spare pool.
> > > > > >> > > This enables us to have lower session timeouts and low
> latency
> > > > > >> mirroring,
> > > > > >> > > in cases where the service providers are OK with having
> spare
> > > > hosts.
> > > > > >> > > This would mean that we would tolerate n consumer members
> > > leaving
> > > > > and
> > > > > >> > > rejoining the group and still provide low latency as long
> as n
> > > <=
> > > > > >> number
> > > > > >> > of
> > > > > >> > > spare consumers.
> > > > > >> > > If there are no spare host available, we can get back to the
> > > idea
> > > > as
> > > > > >> > > described in the KIP.
> > > > > >> > >
> > > > > >> >
> > > > > >> > Thanks,
> > > > > >> >
> > > > > >> > Mayuresh
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> >
> > > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > > >> > konstantine@confluent.io> wrote:
> > > > > >> >
> > > > > >> > > Hi Boyang.
> > > > > >> > >
> > > > > >> > > Thanks for preparing this KIP! It is making good progress
> and
> > > will
> > > > > be
> > > > > >> a
> > > > > >> > > great improvement for stateful Kafka applications.
> > > > > >> > >
> > > > > >> > > Apologies for my late reply, I was away for a while. Lots of
> > > great
> > > > > >> > comments
> > > > > >> > > so far, so I'll probably second most of them in what I
> suggest
> > > > below
> > > > > >> at
> > > > > >> > > this point.
> > > > > >> > >
> > > > > >> > > When I first read the KIP, I wanted to start at the end with
> > > > > something
> > > > > >> > that
> > > > > >> > > wasn't highlighted a lot. That was the topic related to
> > handling
> > > > > >> > duplicate
> > > > > >> > > members. I see now that the initial suggestion of handling
> > this
> > > > > >> situation
> > > > > >> > > during offset commit has been removed, and I agree with
> that.
> > > > Issues
> > > > > >> > > related to membership seem to be handled better when the
> > member
> > > > > joins
> > > > > >> the
> > > > > >> > > group rather than when it tries to commit offsets. This also
> > > > > >> simplifies
> > > > > >> > how
> > > > > >> > > many request types need to change in order to incorporate
> the
> > > new
> > > > > >> member
> > > > > >> > > name field.
> > > > > >> > >
> > > > > >> > > I also agree with what Jason and Guozhang have said
> regarding
> > > > > >> timeouts.
> > > > > >> > > Although semantically, it's easier to think of every
> operation
> > > > > having
> > > > > >> its
> > > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > > >> consolidation
> > > > > >> > > seems preferable here. The definition of embedded protocols
> on
> > > top
> > > > > of
> > > > > >> the
> > > > > >> > > base group membership protocol for rebalancing gives enough
> > > > > >> flexibility
> > > > > >> > to
> > > > > >> > > address such needs in each client component separately.
> > > > > >> > >
> > > > > >> > > Finally, some minor comments:
> > > > > >> > > In a few places the new/proposed changes are referred to as
> > > > > "current".
> > > > > >> > > Which is a bit confusing considering that there is a
> protocol
> > in
> > > > > place
> > > > > >> > > already, and by "current" someone might understand the
> > existing
> > > > one.
> > > > > >> I'd
> > > > > >> > > recommend using new/proposed or equivalent when referring to
> > > > changes
> > > > > >> > > introduced with KIP-345 and current/existing or equivalent
> > when
> > > > > >> referring
> > > > > >> > > to existing behavior.
> > > > > >> > >
> > > > > >> > > There's the following sentence in the "Public Interfaces"
> > > section:
> > > > > >> > > "Since for many stateful consumer/stream applications, the
> > state
> > > > > >> > shuffling
> > > > > >> > > is more painful than short time partial unavailability."
> > > > > >> > > However, my understanding is that the changes proposed with
> > > > KIP-345
> > > > > >> will
> > > > > >> > > not exploit any partial availability. A suggestion for
> dealing
> > > > with
> > > > > >> > > temporary imbalances has been made in "Incremental
> Cooperative
> > > > > >> > Rebalancing"
> > > > > >> > > which can work well with KIP-345, but here I don't see
> > proposed
> > > > > >> changes
> > > > > >> > > that suggest that some resources (e.g. partitions) will keep
> > > being
> > > > > >> used
> > > > > >> > > while others will not be utilized. Thus, you might want to
> > > adjust
> > > > > this
> > > > > >> > > sentence. Correct me if I'm missing something related to
> that.
> > > > > >> > >
> > > > > >> > > In the rejected alternatives, under point 2) I read "we can
> > copy
> > > > the
> > > > > >> > member
> > > > > >> > > id to the config files". I believe it means to say "member
> > name"
> > > > > >> unless
> > > > > >> > I'm
> > > > > >> > > missing something about reusing member ids. Also below I
> read:
> > > "By
> > > > > >> > allowing
> > > > > >> > > consumers to optionally specifying a member id" which
> probably
> > > > > implies
> > > > > >> > > "member name" again. In a sense this section highlights a
> > > > potential
> > > > > >> > > confusion between member name and member id. I wonder if we
> > > could
> > > > > >> come up
> > > > > >> > > with a better term for the new field. StaticTag,
> StaticLabel,
> > or
> > > > > even
> > > > > >> > > StaticName are some suggestions that could potentially help
> > with
> > > > > >> > confusion
> > > > > >> > > between MemberId and MemberName and what corresponds to
> what.
> > > But
> > > > I
> > > > > >> > > wouldn't like to disrupt the discussion with naming
> > conventions
> > > > too
> > > > > >> much
> > > > > >> > at
> > > > > >> > > this point. I just mention it here as a thought.
> > > > > >> > >
> > > > > >> > > Looking forward to see the final details of this KIP. Great
> > work
> > > > so
> > > > > >> far!
> > > > > >> > >
> > > > > >> > > Konstantine
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > > bchen11@outlook.com>
> > > > > >> wrote:
> > > > > >> > >
> > > > > >> > > > Thanks Guozhang for the great summary here, and I have
> been
> > > > > >> following
> > > > > >> > up
> > > > > >> > > > the action items here.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > > timeout
> > > > > and
> > > > > >> > > > registration timeout. Great to see them being addressed in
> > > > client
> > > > > >> side!
> > > > > >> > > >   2.  I double checked the design and I believe that it is
> > ok
> > > to
> > > > > >> have
> > > > > >> > > both
> > > > > >> > > > static member and dynamic member co-exist in the same
> group.
> > > So
> > > > > the
> > > > > >> > > upgrade
> > > > > >> > > > shouldn't be destructive and we are removing the two
> > > membership
> > > > > >> > protocol
> > > > > >> > > > switching APIs.
> > > > > >> > > >   3.  I only have question about this one. I'm still
> reading
> > > the
> > > > > >> > > KafkaApis
> > > > > >> > > > code here. Should I just use the same authorization logic
> > for
> > > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > > > suggested,
> > > > > >> > this
> > > > > >> > > > feature could be better addressed in a separate KIP
> because
> > it
> > > > is
> > > > > >> > pretty
> > > > > >> > > > independent. I could start drafting the KIP once the
> current
> > > > > >> proposal
> > > > > >> > is
> > > > > >> > > > approved.
> > > > > >> > > >   5.  I believe that we don't need fencing in offset
> commit
> > > > > request,
> > > > > >> > > since
> > > > > >> > > > duplicate member.name issue could be handled by join
> group
> > > > > >> request. We
> > > > > >> > > > shall reject join group with known member name but no
> member
> > > id
> > > > > >> (which
> > > > > >> > > > means we already have an active member using this
> identity).
> > > > > >> > > >   6.  I agree to remove that internal config once we move
> > > > forward
> > > > > >> with
> > > > > >> > > > static membership. And I already removed the entire
> section
> > > from
> > > > > the
> > > > > >> > KIP.
> > > > > >> > > >
> > > > > >> > > > Let me know if you have other concerns.
> > > > > >> > > >
> > > > > >> > > > Best,
> > > > > >> > > > Boyang
> > > > > >> > > > ________________________________
> > > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > > >> > > > To: dev
> > > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > > rebalances
> > > > > >> by
> > > > > >> > > > specifying member id
> > > > > >> > > >
> > > > > >> > > > Hello Boyang,
> > > > > >> > > >
> > > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > > appreciate
> > > > > >> your
> > > > > >> > > > patience answering to the feedbacks from the community.
> I'd
> > > like
> > > > > to
> > > > > >> add
> > > > > >> > > my
> > > > > >> > > > 2cents here:
> > > > > >> > > >
> > > > > >> > > > 1. By introducing another two timeout configs,
> > > > > registration_timeout
> > > > > >> and
> > > > > >> > > > expansion_timeout, we are effectively having four timeout
> > > > configs:
> > > > > >> > > session
> > > > > >> > > > timeout, rebalance timeout (configured as "
> > > max.poll.interval.ms
> > > > "
> > > > > on
> > > > > >> > > client
> > > > > >> > > > side), and these two. Interplaying these timeout configs
> can
> > > be
> > > > > >> quite
> > > > > >> > > hard
> > > > > >> > > > for users with such complexity, and hence I'm wondering if
> > we
> > > > can
> > > > > >> > > simplify
> > > > > >> > > > the situation with as less possible timeout configs as
> > > possible.
> > > > > >> Here
> > > > > >> > is
> > > > > >> > > a
> > > > > >> > > > concrete suggestion I'd like propose:
> > > > > >> > > >
> > > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> > addition
> > > > to
> > > > > >> the
> > > > > >> > > > session_timeout for static members, we can just reuse the
> > > > > >> > session_timeout
> > > > > >> > > > and ask users to set it to a larger value when they are
> > > > upgrading
> > > > > a
> > > > > >> > > dynamic
> > > > > >> > > > client to a static client by setting the "member.name" at
> > the
> > > > > same
> > > > > >> > time.
> > > > > >> > > > By
> > > > > >> > > > default, the broker-side min.session.timeout is 6 seconds
> > and
> > > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable
> to
> > me
> > > > (we
> > > > > >> can
> > > > > >> > of
> > > > > >> > > > course modify this broker config to enlarge the valid
> > interval
> > > > if
> > > > > we
> > > > > >> > want
> > > > > >> > > > in practice). And then we should also consider removing
> the
> > > > > >> condition
> > > > > >> > for
> > > > > >> > > > marking a client as failed if the rebalance timeout has
> > > reached
> > > > > >> while
> > > > > >> > the
> > > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > > session_timeout
> > > > > >> > and
> > > > > >> > > > rebalance_timeout are totally separated: the former is
> only
> > > used
> > > > > to
> > > > > >> > > > determine if a consumer member of the group should be
> marked
> > > as
> > > > > >> failed
> > > > > >> > > and
> > > > > >> > > > kicked out of the group, and the latter is only used to
> > > > determine
> > > > > >> the
> > > > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > > > phase.
> > > > > In
> > > > > >> > > other
> > > > > >> > > > words if a member did not send the JoinGroup in time of
> the
> > > > > >> > > > rebalance_timeout, we still include it in the new
> generation
> > > of
> > > > > the
> > > > > >> > group
> > > > > >> > > > and use its old subscription info to send to leader for
> > > > > assignment.
> > > > > >> > Later
> > > > > >> > > > if the member came back with HeartBeat request, we can
> still
> > > > > follow
> > > > > >> the
> > > > > >> > > > normal path to bring it to the latest generation while
> > > checking
> > > > > that
> > > > > >> > its
> > > > > >> > > > sent JoinGroup request contains the same subscription info
> > as
> > > we
> > > > > >> used
> > > > > >> > to
> > > > > >> > > > assign the partitions previously (which should be likely
> the
> > > > case
> > > > > in
> > > > > >> > > > practice). In addition, we should let static members to
> not
> > > send
> > > > > the
> > > > > >> > > > LeaveGroup request when it is gracefully shutdown, so
> that a
> > > > > static
> > > > > >> > > member
> > > > > >> > > > can only be leaving the group if its session has timed
> out,
> > OR
> > > > it
> > > > > >> has
> > > > > >> > > been
> > > > > >> > > > indicated to not exist in the group any more (details
> > below).
> > > > > >> > > >
> > > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > > Cooperative
> > > > > >> > > > Rebalancing, in which we will encode the "when to
> rebalance"
> > > > logic
> > > > > >> at
> > > > > >> > the
> > > > > >> > > > application level, instead of at the protocol level. By
> > doing
> > > > this
> > > > > >> we
> > > > > >> > can
> > > > > >> > > > also enable a few other optimizations, e.g. at the Streams
> > > level
> > > > > to
> > > > > >> > first
> > > > > >> > > > build up the state store as standby tasks and then
> trigger a
> > > > > second
> > > > > >> > > > rebalance to actually migrate the active tasks while
> keeping
> > > the
> > > > > >> actual
> > > > > >> > > > rebalance latency and hence unavailability window to be
> > small
> > > (
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=N8BTmhWAmDGJc9%2BQl6ulM9Qa5vzxIyXaGzCDILSIehs%3D&amp;reserved=0
> > > > > >> > > ).
> > > > > >> > > > I'd propose we align
> > > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > > >> > expansion_timeout
> > > > > >> > > as
> > > > > >> > > > part of the protocol layer, but only do that at the
> > > > application's
> > > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We
> can
> > > > > still,
> > > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > > >> > > > <
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=VO0%2F5TczxUBuJkK7NurBwa1X0wIXwm0WHx4jrCzY0%2Fo%3D&amp;reserved=0
> > > > > >> > > >*"
> > > > > >> > > > though as part of this KIP
> > > > > >> > > > since we have discussed about its limit and think it is
> > > actually
> > > > > >> not a
> > > > > >> > > very
> > > > > >> > > > good design and could be replaced with client-side logic
> > > above.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> > this
> > > > KIP.
> > > > > >> More
> > > > > >> > > > specifically, let's say after we have upgraded broker
> > version
> > > to
> > > > > be
> > > > > >> > able
> > > > > >> > > to
> > > > > >> > > > recognize the new versions of JoinGroup request and the
> > admin
> > > > > >> requests,
> > > > > >> > > how
> > > > > >> > > > should we upgrade the clients and enable static groups? On
> > top
> > > > of
> > > > > my
> > > > > >> > head
> > > > > >> > > > if we do a rolling bounce in which we set the member.name
> > > > config
> > > > > as
> > > > > >> > well
> > > > > >> > > > as
> > > > > >> > > > optionally increase the session.timeout config when we
> > bounce
> > > > each
> > > > > >> > > > instance, then during this rolling bounces we will have a
> > > group
> > > > > >> > contained
> > > > > >> > > > with both dynamic members and static members. It means
> that
> > we
> > > > > >> should
> > > > > >> > > have
> > > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > > JoinGroup
> > > > > >> > > requests
> > > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > > member.id"
> > > > > >> > > mapping
> > > > > >> > > > will only be partial at this scenario. Also could you
> > describe
> > > > if
> > > > > >> the
> > > > > >> > > > upgrade to the first version that support this feature
> would
> > > > ever
> > > > > >> get
> > > > > >> > any
> > > > > >> > > > benefits, or only the future upgrade path for rolling
> > bounces
> > > > > could
> > > > > >> get
> > > > > >> > > > benefits out of this feature?
> > > > > >> > > >
> > > > > >> > > > If that's the case and we will do 1) as suggested above,
> do
> > we
> > > > > still
> > > > > >> > need
> > > > > >> > > > the enableStaticMembership and enableDynamicMembership
> admin
> > > > > >> requests
> > > > > >> > any
> > > > > >> > > > more? Seems it is not necessary any more as we will only
> > have
> > > > the
> > > > > >> > notion
> > > > > >> > > of
> > > > > >> > > > "dynamic or static members" that can co-exist in a group
> > while
> > > > > >> there no
> > > > > >> > > > notion of "dynamic or static groups", and hence these two
> > > > requests
> > > > > >> are
> > > > > >> > > not
> > > > > >> > > > needed anymore.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > 3. We need to briefly talk about the implications for ACL
> as
> > > we
> > > > > >> > introduce
> > > > > >> > > > new admin requests that are related to a specific
> group.id.
> > > For
> > > > > >> > example,
> > > > > >> > > > we
> > > > > >> > > > need to make sure that whoever created the group or joined
> > the
> > > > > group
> > > > > >> > can
> > > > > >> > > > actually send admin requests for the group, otherwise the
> > > > > >> application
> > > > > >> > > > owners need to bother the Kafka operators on a
> multi-tenant
> > > > > cluster
> > > > > >> > every
> > > > > >> > > > time they want to send any admin requests for their groups
> > > which
> > > > > >> would
> > > > > >> > be
> > > > > >> > > > an operational nightmare.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > 4. I like Jason's suggestion of adding an optional field
> for
> > > the
> > > > > >> list
> > > > > >> > of
> > > > > >> > > > member names, and I'm wondering if that can be done as
> part
> > of
> > > > the
> > > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > > members,
> > > > > we
> > > > > >> > will
> > > > > >> > > > enforce a rebalance immediately since it indicates that
> some
> > > > > static
> > > > > >> > > member
> > > > > >> > > > will be officially kicked out of the group and some new
> > static
> > > > > >> members
> > > > > >> > > may
> > > > > >> > > > be added. So back to 1.a) above, a static member can only
> be
> > > > > kicked
> > > > > >> out
> > > > > >> > > of
> > > > > >> > > > the group if a) its session (arguably long period of time)
> > has
> > > > > timed
> > > > > >> > out,
> > > > > >> > > > and b) this admin request explicitly state that it is no
> > > longer
> > > > > >> part of
> > > > > >> > > the
> > > > > >> > > > group. As for execution I'm fine with keeping it as a
> future
> > > > work
> > > > > of
> > > > > >> > this
> > > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > > >> > > >
> > > > > >> > > > Following are minor comments:
> > > > > >> > > >
> > > > > >> > > > 5. I'm not sure if we need to include "member.name" as
> part
> > > of
> > > > > the
> > > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > > memberId
> > > > > >> plus
> > > > > >> > > the
> > > > > >> > > > generation number should be sufficient for fencing even
> with
> > > > > static
> > > > > >> > > > members.
> > > > > >> > > >
> > > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid
> > of
> > > > the
> > > > > "
> > > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > Guozhang
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> > lindong28@gmail.com
> > > >
> > > > > >> wrote:
> > > > > >> > > >
> > > > > >> > > > > Hey Boyang,
> > > > > >> > > > >
> > > > > >> > > > > Thanks for the proposal! This is very useful. I have
> some
> > > > > comments
> > > > > >> > > below:
> > > > > >> > > > >
> > > > > >> > > > > 1) The motivation currently explicitly states that the
> > goal
> > > is
> > > > > to
> > > > > >> > > improve
> > > > > >> > > > > performance for heavy state application. It seems that
> the
> > > > > >> motivation
> > > > > >> > > can
> > > > > >> > > > > be stronger with the following use-case. Currently for
> > > > > MirrorMaker
> > > > > >> > > > cluster
> > > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> > > time
> > > > to
> > > > > >> > > rolling
> > > > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > > > process
> > > > > >> > restart
> > > > > >> > > > > will trigger a rebalance which currently pause the
> > > consumption
> > > > > of
> > > > > >> the
> > > > > >> > > all
> > > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > > stated
> > > > in
> > > > > >> this
> > > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > > specified
> > > > > >> > > timeout
> > > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > > rebalance
> > > > > >> > (e.g.
> > > > > >> > > > for
> > > > > >> > > > > leader restart) for the entire rolling bounce, which
> will
> > > > > >> > significantly
> > > > > >> > > > > improves the availability of the MirrorMaker pipeline.
> In
> > my
> > > > > >> opinion,
> > > > > >> > > the
> > > > > >> > > > > main benefit of the KIP is to avoid unnecessary
> rebalance
> > if
> > > > the
> > > > > >> > > consumer
> > > > > >> > > > > process can be restarted within soon, which helps
> > > performance
> > > > > >> even if
> > > > > >> > > > > overhead of state shuffling for a given process is
> small.
> > > > > >> > > > >
> > > > > >> > > > > 2) In order to simplify the KIP reading, can you follow
> > the
> > > > > >> writeup
> > > > > >> > > style
> > > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> > > such
> > > > as
> > > > > >> new
> > > > > >> > > > > configs (e.g. registration timeout), new
> request/response,
> > > new
> > > > > >> > > > AdminClient
> > > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > > Currently
> > > > > >> some
> > > > > >> > > of
> > > > > >> > > > > these are specified in the Proposed Change section which
> > > makes
> > > > > it
> > > > > >> a
> > > > > >> > bit
> > > > > >> > > > > inconvenient to understand the new interface that will
> be
> > > > > exposed
> > > > > >> to
> > > > > >> > > > user.
> > > > > >> > > > > Explanation of the current two-phase rebalance protocol
> > > > probably
> > > > > >> can
> > > > > >> > be
> > > > > >> > > > > moved out of public interface section.
> > > > > >> > > > >
> > > > > >> > > > > 3) There are currently two version of JoinGroupRequest
> in
> > > the
> > > > > KIP
> > > > > >> and
> > > > > >> > > > only
> > > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > > >> > > > >
> > > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to
> force
> > > > > >> rebalance
> > > > > >> > > could
> > > > > >> > > > > be helpful here, but we will make a call once we
> finished
> > > the
> > > > > >> major
> > > > > >> > > > > implementation". So this seems to be still an open
> > question
> > > in
> > > > > the
> > > > > >> > > > current
> > > > > >> > > > > design. We probably want to agree on this before voting
> > for
> > > > the
> > > > > >> KIP.
> > > > > >> > > > >
> > > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > > consumer.
> > > > > Can
> > > > > >> > you
> > > > > >> > > > > specify the name of the config key and the default
> config
> > > > value?
> > > > > >> > > Possible
> > > > > >> > > > > default values include empty string or null (similar to
> > > > > >> > transaction.id
> > > > > >> > > > in
> > > > > >> > > > > producer config).
> > > > > >> > > > >
> > > > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > > > persist
> > > > > >> > member
> > > > > >> > > > > name map, currently if consumer coordinator broker goes
> > > > offline,
> > > > > >> > > > rebalance
> > > > > >> > > > > is triggered and consumers will try connect to the new
> > > > > >> coordinator.
> > > > > >> > If
> > > > > >> > > > > these consumers can connect to the new coordinator
> within
> > > > > >> > > > > max.poll.interval.ms which by default is 5 minutes,
> given
> > > > that
> > > > > >> > broker
> > > > > >> > > > can
> > > > > >> > > > > use a deterministic algorithm to determine the partition
> > ->
> > > > > >> > member_name
> > > > > >> > > > > mapping, each consumer should get assigned the same set
> of
> > > > > >> partitions
> > > > > >> > > > > without requiring state shuffling. So it is not clear
> > > whether
> > > > we
> > > > > >> > have a
> > > > > >> > > > > strong use-case for this new logic. Can you help clarify
> > > what
> > > > is
> > > > > >> the
> > > > > >> > > > > benefit of using topic "static_member_map" to persist
> > member
> > > > > name
> > > > > >> > map?
> > > > > >> > > > >
> > > > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > > > config,
> > > > > >> it is
> > > > > >> > > > > mentioned that "we are using expansion timeout to
> replace
> > > > > >> rebalance
> > > > > >> > > > > timeout, which is configured by max.poll.intervals from
> > > client
> > > > > >> side,
> > > > > >> > > and
> > > > > >> > > > > using registration timeout to replace session timeout".
> > > > > Currently
> > > > > >> the
> > > > > >> > > > > default max.poll.interval.ms is configured to be 5
> > minutes
> > > > and
> > > > > >> there
> > > > > >> > > > will
> > > > > >> > > > > be only one rebalance if all new consumers can join
> > within 5
> > > > > >> minutes.
> > > > > >> > > So
> > > > > >> > > > it
> > > > > >> > > > > is not clear whether we have a strong use-case for this
> > new
> > > > > >> config.
> > > > > >> > Can
> > > > > >> > > > you
> > > > > >> > > > > explain what is the benefit of introducing this new
> > config?
> > > > > >> > > > >
> > > > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > > > version
> > > > > >> of
> > > > > >> > > > > protocol, we will also increase the join group request
> > > version
> > > > > to
> > > > > >> v4
> > > > > >> > > when
> > > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not
> the
> > > > latest
> > > > > >> (<
> > > > > >> > > v4),
> > > > > >> > > > > the join group request shall be downgraded to v3 without
> > > > setting
> > > > > >> the
> > > > > >> > > > member
> > > > > >> > > > > Id". It is probably simpler to just say that this
> feature
> > is
> > > > > >> enabled
> > > > > >> > if
> > > > > >> > > > > JoinGroupRequest V4 is supported on both client and
> broker
> > > and
> > > > > >> > > > MEMBER_NAME
> > > > > >> > > > > is configured with non-empty string.
> > > > > >> > > > >
> > > > > >> > > > > 9) It is mentioned that broker may return
> > > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > > >> > > error
> > > > > >> > > > > in OffsetCommitResponse for "commit requests under
> static
> > > > > >> > membership".
> > > > > >> > > > Can
> > > > > >> > > > > you clarify how broker determines whether the commit
> > request
> > > > is
> > > > > >> under
> > > > > >> > > > > static membership?
> > > > > >> > > > >
> > > > > >> > > > > Thanks,
> > > > > >> > > > > Dong
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > --
> > > > > >> > > > -- Guozhang
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> >
> > > > > >> > --
> > > > > >> > -Regards,
> > > > > >> > Mayuresh R. Gharat
> > > > > >> > (862) 250-7125
> > > > > >> >
> > > > > >>
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -Regards,
> > > > > > Mayuresh R. Gharat
> > > > > > (862) 250-7125
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
> >
> > --
> > -- Guozhang
> >
>
>
> --
> -- Guozhang
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
In fact I feel that it's more convenient for user to specify a list of instance id prefixes. Because
for general consumer application we couldn't always find a proper prefix to remove a list of consumers.
So we are either adding list[instanceid prefix], or we could add two fields: instanceid prefix, and list[instanceid]
for clarity purpose. As you know, two options are equivalent since full name is subset of prefix.

Let me know your thoughts!

Boyang
________________________________
From: Boyang Chen <bc...@outlook.com>
Sent: Thursday, November 29, 2018 3:39 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Thanks Guozhang for the new proposal here!

So I'd like to propose a slightly modified version of LeaveGroupRequest:
instead of letting the static member consumer client themselves to send the
request (which means we still need to have some hidden configs to turn it
off like we did today), how about just letting any other client to send
this request since the LeaveGroupRequest only requires group.id and
member.id? So back to your operational scenarios, if some static member has
been found crashed and it is not likely to comeback, or we simply want to
shrink the size of the group by shutting down some static members, we can
use an admin client to send the LeaveGroupRequest after the instance has
been completely shutdown or crashed to kick them out of the group and also
triggers the rebalance.

One issue though, is that users may not know the member id required in the
LeaveGroupRequest. To work around it we can add the `group.instance.id`
along with the member id as well and then allow member id null-able. The
coordinator logic would then be modified as 1) if member.id is specified,
ignore instance.id and always use member.id to find the member to kick out,
2) otherwise, try with the instance.id to find the corresponding member.id
and kick it out, 3) if none is found, reject with an error code.

So in sum the alternative changes are:

a) Modify LeaveGroupRequest to add group.instance.id
b) Modify coordinator logic to handle such request on the broker side.
c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
instanceId)" which will be translated as a LeaveGroupRequest.
d) [Optional] we can even batch the request by allowing
"removeMemberFromGroup(groupId, list[instanceId])" and then make `member.id`
and `instance.id` field of LeaveGroupRequest to be an array instead of a
single entry.
e) We can also remove the admin ConsumerRebalanceRequest as well for
simplicity (why not? paranoid of having as less request protocols as
possible :), as it is not needed anymore with the above proposal.
I agree that reusing LeaveGroupRequest is actually a good idea: we only need to iterate
over an existing request format. Also I found that we haven't discussed how we want to enable
this feature on Streaming applications, which is different from common consumer application in that
Stream app uses stream thread as individual consumer.
For example if user specifies the client id, the stream consumer client id will be like:
User client id + "-StreamThread-" + thread id + "-consumer"

So I'm thinking we should do sth similar for defining group.instance.id on Stream. We shall define another
config called `stream.instance.id` which would be used as prefix, and for each thread consumer the formula
will look like:
`group.instance.id` = `stream.instance.id` + "-" + thread id + "-consumer"

And for the ease of use, the interface of leave group request could include `group.instance.id.prefix` instead of
`group.instance.id` so that we could batch remove consumers relating to a single stream instance. This is more intuitive
and flexible since specifying names of 16~32 * n (n = number of stream instances to shut down) consumers is not an easy
job without client management tooling.

How does this workaround sound?

Boyang
________________________________
From: Guozhang Wang <wa...@gmail.com>
Sent: Thursday, November 29, 2018 2:38 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

I was thinking that with the optional static members in the admin
ConsumerRebalanceRequest it should be sufficient to kick out the static
member before their session timeout (arguably long in practice) have not
reached. But now I see your concern is that in some situations the admin
operators may not even know the full list of static members, but ONLY know
which static member has failed and hence would like to kick out of the
group.

So I'd like to propose a slightly modified version of LeaveGroupRequest:
instead of letting the static member consumer client themselves to send the
request (which means we still need to have some hidden configs to turn it
off like we did today), how about just letting any other client to send
this request since the LeaveGroupRequest only requires group.id and
member.id? So back to your operational scenarios, if some static member has
been found crashed and it is not likely to comeback, or we simply want to
shrink the size of the group by shutting down some static members, we can
use an admin client to send the LeaveGroupRequest after the instance has
been completely shutdown or crashed to kick them out of the group and also
triggers the rebalance.

One issue though, is that users may not know the member id required in the
LeaveGroupRequest. To work around it we can add the `group.instance.id`
along with the member id as well and then allow member id null-able. The
coordinator logic would then be modified as 1) if member.id is specified,
ignore instance.id and always use member.id to find the member to kick out,
2) otherwise, try with the instance.id to find the corresponding member.id
and kick it out, 3) if none is found, reject with an error code.

So in sum the alternative changes are:

a) Modify LeaveGroupRequest to add group.instance.id
b) Modify coordinator logic to handle such request on the broker side.
c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
instanceId)" which will be translated as a LeaveGroupRequest.
d) [Optional] we can even batch the request by allowing
"removeMemberFromGroup(groupId, list[instanceId])" and then make `member.id`
and `instance.id` field of LeaveGroupRequest to be an array instead of a
single entry.
e) We can also remove the admin ConsumerRebalanceRequest as well for
simplicity (why not? paranoid of having as less request protocols as
possible :), as it is not needed anymore with the above proposal.


WDYT?


Guozhang

On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Guozhang and Mayuresh for the follow up! Answers are listed below.
>
>
> >  5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
>
> Great question. Let's use c1~c4 example here:
>
>   1.  Consumer c1, c2, c3, c4 in stable state
>   2.  c4 goes down and we detect this issue before session timeout through
> client monitoring. Initiate a ConsumerRebalanceRequest.
>   3.  A rebalance will be kicking off, and after rebalance timeout we
> shall keep the same assignment for c1~4, if the session timeout for c4
> hasn't reached
>   4.  Group back to stable with c1~4 (although c4 is actually offline)
>   5.  c4 session timeout finally reached: another rebalance triggered.
>
> For step 3, if session timeout triggered within rebalance timeout, only
> c1~3 will be participating in the rebalance. This is what we mean by saying
> "rebalance
> timeout shall not remove current members, only session timeout will do."
> As you could see this is not an ideal scenario: we trigger extra rebalance
> at step 5. In my reply to Guozhang I'm asking whether we should still use
> LeaveGroupRequest for static members to send a signal to broker saying "I'm
> currently offline", and when we send ConsumerRebalanceRequest to broker, we
> will actually kick off c4 because it says it's offline already, saving one
> or multiple additional rebalances later. This way the
> ConsumerRebalanceRequest will be more effective in making correct judgement
> on the group status since we have more feedback from client side.
>
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> Previous proposal should be suffice to answer this question 😊
>
> - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> Thanks for catching it! And yes, we shall use one extra round-trip between
> consumer
> and broker to inform the new member id allocation.
>
> Next is the replies to Guozhang's comment:
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
> I'm also in favor of storing all the in-group static members. In fact we
> could reuse
> the static membership mapping to store this information. Do you think
> that we should let static member send leave group request to indicate
> their status of "leaving",
> and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> suggesting we should
> remove those members when kicking off rebalance since we are shutting them
> down already.
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
> Sure we have enough votes for ids 😊I will finalize the name to `
> group.instance.id`, does that
> sound good?
>
> Best,
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Wednesday, November 28, 2018 4:51 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Regarding Jason's question and Boyang's responses:
>
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
>
>
> Guozhang
>
>
> On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Thanks for the replies. Please find the follow up queries below.
> >
> >     5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
> > The plan is to disable member kick out when rebalance.timeout is reached,
> > so basically we are not "waiting" any
> > join group request from existing members; we shall just rebalance base on
> > what we currently have within the group
> > metadata. Lagging consumer will trigger rebalance later if session
> timeout
> > > rebalance timeout.
> >
> > >
> > Just wanted to understand this better. Lets take an example, say we have
> a
> > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > Everything is running fine and suddenly C4 host has issues and it goes
> > > down. Now we notice that we can still operate with c1, c2, c3 and don't
> > > want to wait for
> > > c4 to come back up. We use the admin api
> > > "invokeConsumerRebalance("GroupA")".
> > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > > group again (in there heartBeatResponse) as first step of rebalance.
> > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> > > delayed. At this stage, if we are not "waiting" on any join group
> > request,
> > > few things can happen :
> > >
> > >    - c4's partitions are distributed only among c1,c2. c3 maintains its
> > >    original assignment. c1, c2 will start processing the newly assigned
> > >    partitions.
> > >
> > > OR
> > >
> > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
> > >    processing the newly assigned partitions. c3 gets to know about the
> > newly
> > >    assigned partitions later when it sends the JoinGroupRequest (which
> > was
> > >    delayed).
> > >
> > > OR
> > >
> > >    - Will the rebalance do a complete reassignment, where c1, c2, c3
> have
> > >    to give up there partitions and all the partitions belonging to c1,
> > c2, c3,
> > >    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
> > >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> > revoke
> > >    there partitions and rejoin the group.
> > >
> > > This is as per my understanding of how the KIP would work without
> > changing
> > > the underlying group coordination workflow. Please correct me if I
> > > misunderstood something here.
> > >
> >
> >
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> >
> > - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=JvM8EWgbyIouukmr%2FE9uuW%2BF96Nbc8TKl%2BwRu9IUwDM%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> both
> > > in this reply.
> > >
> > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > >
> > > No, we shall only rely on broker to allocate member.id for the
> consumer
> > > instances. FYI, I already
> > >
> > > started the discussion thread for KIP-394 😊
> > >
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > >
> > > Yes, we could only use unknown member id. Actually this part of the
> > > proposal is outdated,
> > >
> > > let me do another audit of the whole doc. Basically, it is currently
> > > impossible to send `member.id`
> > >
> > > when consumer restarted. Sorry for the confusions!
> > >
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > >
> > > I would take Jason's advice that each time we have unknown member
> joining
> > > the group, the broker will
> > >
> > > always assign a new and unique id to track its identity. In this way,
> > > consumer with duplicate member name
> > >
> > > will be fenced.
> > >
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > >
> > > Like explained above, we shall not materialize the member.id. Instead
> we
> > > need to rely on broker to allocate
> > >
> > > a unique id for consumer just like what we have now.
> > >
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > >
> > > The plan is to disable member kick out when rebalance.timeout is
> reached,
> > > so basically we are not "waiting" any
> > >
> > > join group request from existing members; we shall just rebalance base
> on
> > > what we currently have within the group
> > >
> > > metadata. Lagging consumer will trigger rebalance later if session
> > timeout
> > > > rebalance timeout.
> > >
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > >
> > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > >
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > >
> > > Nah, I'm gonna remove it because it seems causing more confusion than
> > > making my assumption clear, which is
> > >
> > > "there could be other failure cases that I can't enumerate now" 😊
> > >
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > >
> > > Great suggestion! Added a simple rollback plan.
> > >
> > >
> > > Next is answering Jason's suggestions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > Yep, I like this idea! Keep giving out refresh member.id when facing
> > > anonymous request will definitely
> > >
> > > prevent processing bug due to duplicate consumers, however I don't
> think
> > I
> > > fully understand the 3rd paragraph where
> > >
> > > you mentioned  "There needs to be an expectation for static members
> that
> > > the member.id of a static member will
> > >
> > > not be changed except when a new member with the same member.name
> joins
> > > the group. "  How do you plan
> > > to know whether this member is new member or old member? I feel even
> with
> > > zombie consumer takes the ownership,
> > > it should be detected very quickly (as MISMATCH_ID exception trigger
> > > original consumer instance dies)
> > > and end user will start to fix it right away. Is there any similar
> logic
> > > we applied in fencing duplicate `transaction.id`?
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > This API is needed when we need to immediately trigger rebalance
> instead
> > > of waiting session timeout
> > >
> > > or rebalance timeout (Emergent scale up/down). It is very necessary to
> > > have it for
> > >
> > > management purpose because user could choose when to trigger rebalance
> > > pretty freely,
> > >
> > > gaining more client side control.
> > >
> > > In the meanwhile I see your point that we need to actually have the
> > > ability to kick out members that we plan
> > >
> > > to scale down fast (as rebalance timeout no longer kicks any offline
> > > member out of the group), I will think of adding an optional
> > >
> > > list of members that are ready to be removed.
> > >
> > > Another idea is to let static member send `LeaveGroupRequest` when they
> > > are going offline (either scale down or bouncing),
> > >
> > > and broker will cache this information as "OfflineMembers" without
> > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> will
> > >
> > > kick the static members that are currently offline and trigger
> rebalance
> > > immediately. How does this plan sound?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> > > the namespace idea, and
> > >
> > > since we already use `member.name` in a lot of context, I decide to
> > > rename the config to `group.member.name`
> > >
> > > which should be sufficient for solving all the concerns we have now.
> > > Sounds good?
> > >
> > >
> > > Thank you for your great suggestions! Let me know if my reply makes
> sense
> > > her.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Jason Gustafson <ja...@confluent.io>
> > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the updates. Looks like we're headed in the right direction
> > and
> > > clearly the interest that this KIP is receiving shows how strong the
> > > motivation is!
> > >
> > > I have a few questions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks a lot for replying to all the queries and discussions here, so
> > > > patiently.
> > > > Really appreciate it.
> > > >
> > > > Had a few questions and suggestions after rereading the current
> version
> > > of
> > > > the KIP :
> > > >
> > > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=1dQUbfyutqmSN58If1NisiS4Momk4Ri6v9B6DPS7bno%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > gharatmayuresh15@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > > >
> > > > >> Hey Dong, thanks for the follow-up here!
> > > > >>
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >> Great question. I feel `member.name` is enough to explain itself,
> > it
> > > > >> seems not very
> > > > >> helpful to make the config name longer. Comparing `name` with `id`
> > > gives
> > > > >> user the
> > > > >> impression that they have the control over it with customized rule
> > > than
> > > > >> library decided.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >> We are not changing the default session timeout value. We are just
> > > > >> changing the
> > > > >> cap we are enforcing on the session timeout max value. So this
> > change
> > > is
> > > > >> not affecting
> > > > >> what kind of membership end user is using, and loosing the cap is
> > > giving
> > > > >> end user
> > > > >> more flexibility on trade-off between liveness and stability.
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> because
> > > with
> > > > >> the KIP-394<
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=E3cqYTMRFsAs5TQI4JxHm3kOWCfkVWjpuc%2BuNHezwG0%3D&amp;reserved=0
> > > > >> >
> > > > >> we will automatically fence all join requests with
> > UNKNOWN_MEMBER_ID.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >> Same answer as question 3). This part of the logic shall be
> removed
> > > from
> > > > >> the proposal.
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >> Sounds good! Added both methods.
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > >> .
> > > > >> Added the script.
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >> Sounds good. Right now I feel for both static and dynamic
> membership
> > > it
> > > > is
> > > > >> more manageable to introduce the consumer rebalance method through
> > > admin
> > > > >> client API.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Sure, I will add more details to the API.
> > > > >>
> > > > >>
> > > > >> Thanks again for the helpful suggestions!
> > > > >>
> > > > >>
> > > > >> Best,
> > > > >> Boyang
> > > > >>
> > > > >> ________________________________
> > > > >> From: Dong Lin <li...@gmail.com>
> > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > >> To: dev
> > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > > >> specifying member id
> > > > >>
> > > > >> Hey Boyang,
> > > > >>
> > > > >> Thanks for the update! Here are some followup comments:
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=gWbzMbM%2ByA8%2FGbeC5Eh6kt8FuE5j%2FnrHaKE%2FhipcCBQ%3D&amp;reserved=0
> > > > >> .
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Thanks,
> > > > >> Dong
> > > > >>
> > > > >>
> > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > > > wrote:
> > > > >>
> > > > >> > Hey Mayuresh,
> > > > >> >
> > > > >> >
> > > > >> > thanks for your feedbacks! I will try do another checklist here.
> > > > >> >
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> >
> > > > >> > Yep it's true, we will prevent client from sending leave group
> > > request
> > > > >> > when they are set with `member.name`.
> > > > >> >
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > Good question, we will only kick off member due to session
> timeout
> > > > >> within
> > > > >> > static membership. Let me update the KIP to clearly assert that.
> > > > >> >
> > > > >> > > How is this case (missing member id) handled on the client
> side?
> > > > What
> > > > >> is
> > > > >> > the application that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > I have extended the two exceptions within join group response
> V4.
> > > > >> > Basically I define both corresponding actions to be immediate
> > > failing
> > > > >> > client application, because so far it is unknown what kind of
> > client
> > > > >> issue
> > > > >> > could trigger them. After the first version, we will keep
> enhance
> > > the
> > > > >> error
> > > > >> > handling logic!
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > The goal of extending session timeout cap is for users with good
> > > > client
> > > > >> > side monitoring tools that could auto-heal the dead consumers
> very
> > > > >> fast. So
> > > > >> > it is optional (and personal) to extend session timeout to a
> > > > reasonable
> > > > >> > number with different client scenarios.
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > functionality
> > > > >> > to only use as a timer to
> > > > >> > end prepare rebalance stage. This way, late joining static
> members
> > > > will
> > > > >> > not trigger further rebalance
> > > > >> > as long as they are within session timeout. I added your
> highlight
> > > to
> > > > >> the
> > > > >> > KIP!
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> because
> > we
> > > > >> > believe client side could
> > > > >> > better handle this logic.
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > Great idea! In fact on top of static membership we could later
> > > > introduce
> > > > >> > APIs to set hard-coded
> > > > >> > client ids to the group and replace the dead host, or as you
> > > proposed
> > > > to
> > > > >> > define spare host as
> > > > >> > what I understood as hot backup. I will put both Jason and your
> > > > >> > suggestions into a separate section
> > > > >> > called "Future works". Note that this spare host idea may be
> also
> > > > >> solvable
> > > > >> > through rebalance protocol
> > > > >> > IMO.
> > > > >> >
> > > > >> > Thank you again for the great feedback!
> > > > >> >
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> > email
> > > > on
> > > > >> my
> > > > >> > thread, so I will just do a checklist here!
> > > > >> >
> > > > >> >
> > > > >> > 1) The motivation currently explicitly states that the goal is
> to
> > > > >> improve
> > > > >> >
> > > > >> > performance for heavy state application. It seems that the
> > > motivation
> > > > >> can
> > > > >> >
> > > > >> > be stronger with the following use-case. Currently for
> MirrorMaker
> > > > >> cluster
> > > > >> >
> > > > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > > rolling
> > > > >> >
> > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > > > restart
> > > > >> >
> > > > >> > will trigger a rebalance which currently pause the consumption
> of
> > > the
> > > > >> all
> > > > >> >
> > > > >> > partitions of the MirrorMaker cluster. With the change stated in
> > > this
> > > > >> >
> > > > >> > patch, as long as a MirrorMaker can restart within the specified
> > > > timeout
> > > > >> >
> > > > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> > > (e.g.
> > > > >> for
> > > > >> >
> > > > >> > leader restart) for the entire rolling bounce, which will
> > > > significantly
> > > > >> >
> > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > opinion,
> > > > >> the
> > > > >> >
> > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > > >> consumer
> > > > >> >
> > > > >> > process can be restarted within soon, which helps performance
> even
> > > if
> > > > >> >
> > > > >> > overhead of state shuffling for a given process is small.
> > > > >> >
> > > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > > making
> > > > >> the
> > > > >> > motivation more solid!
> > > > >> >
> > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > writeup
> > > > >> style
> > > > >> > of other KIP (e.g. KIP-98) and list the interface change such as
> > new
> > > > >> > configs (e.g. registration timeout), new request/response, new
> > > > >> AdminClient
> > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> > > some
> > > > of
> > > > >> > these are specified in the Proposed Change section which makes
> it
> > a
> > > > bit
> > > > >> > inconvenient to understand the new interface that will be
> exposed
> > to
> > > > >> user.
> > > > >> > Explanation of the current two-phase rebalance protocol probably
> > can
> > > > be
> > > > >> > moved out of public interface section.
> > > > >> > This is a great suggestion! I just consolidated all the public
> API
> > > > >> > changes, and the whole KIP
> > > > >> > looks much more organized!
> > > > >> >
> > > > >> > 3) There are currently two version of JoinGroupRequest in the
> KIP
> > > and
> > > > >> only
> > > > >> > one of them has field memberId. This seems confusing.
> > > > >> > Yep, I already found this issue and fixed it.
> > > > >> >
> > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > rebalance
> > > > >> could
> > > > >> > be helpful here, but we will make a call once we finished the
> > major
> > > > >> > implementation". So this seems to be still an open question in
> the
> > > > >> current
> > > > >> > design. We probably want to agree on this before voting for the
> > KIP.
> > > > >> > We have finalized the idea that this API is needed.
> > > > >> >
> > > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> Can
> > > you
> > > > >> > specify the name of the config key and the default config value?
> > > > >> Possible
> > > > >> > default values include empty string or null (similar to
> > > > transaction.id<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=qWkxpqJMF6ugtVKhupIthbcmSNFmp4sX5EfgvKAiAQo%3D&amp;reserved=0
> > > > >> >
> > > > >> > in
> > > > >> > producer config).
> > > > >> > I have defined the `member.name` in "New configuration"
> section.
> > > > >> >
> > > > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > > > member
> > > > >> > name map, currently if consumer coordinator broker goes offline,
> > > > >> rebalance
> > > > >> > is triggered and consumers will try connect to the new
> > coordinator.
> > > If
> > > > >> > these consumers can connect to the new coordinator within
> > > > >> > max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > >> >
> > > > >> > which by default is 5 minutes, given that broker can
> > > > >> > use a deterministic algorithm to determine the partition ->
> > > > member_name
> > > > >> > mapping, each consumer should get assigned the same set of
> > > partitions
> > > > >> > without requiring state shuffling. So it is not clear whether we
> > > have
> > > > a
> > > > >> > strong use-case for this new logic. Can you help clarify what is
> > the
> > > > >> > benefit of using topic "static_member_map" to persist member
> name
> > > map?
> > > > >> > I have discussed with Guozhang offline, and I believe reusing
> the
> > > > >> current
> > > > >> > `_consumer_offsets`
> > > > >> > topic is a better and unified solution.
> > > > >> >
> > > > >> > 7) Regarding the introduction of the expensionTimeoutMs config,
> it
> > > is
> > > > >> > mentioned that "we are using expansion timeout to replace
> > rebalance
> > > > >> > timeout, which is configured by max.poll.intervals from client
> > side,
> > > > and
> > > > >> > using registration timeout to replace session timeout".
> Currently
> > > the
> > > > >> > default max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=kRFKKVocKt0U4Vb%2BepPC7xUAZQ4KgUxzJ7%2FxEOqtfwA%3D&amp;reserved=0
> > > > >> >
> > > > >> > is configured to be 5 minutes and there will
> > > > >> > be only one rebalance if all new consumers can join within 5
> > > minutes.
> > > > >> So it
> > > > >> > is not clear whether we have a strong use-case for this new
> > config.
> > > > Can
> > > > >> you
> > > > >> > explain what is the benefit of introducing this new config?
> > > > >> > Previously our goal is to use expansion timeout as a workaround
> > for
> > > > >> > triggering multiple
> > > > >> > rebalances when scaling up members are not joining at the same
> > time.
> > > > It
> > > > >> is
> > > > >> > decided to
> > > > >> > be addressed by client side protocol change, so we will not
> > > introduce
> > > > >> > expansion timeout.
> > > > >> >
> > > > >> > 8) It is mentioned that "To distinguish between previous version
> > of
> > > > >> > protocol, we will also increase the join group request version
> to
> > v4
> > > > >> when
> > > > >> > MEMBER_NAME is set" and "If the broker version is not the latest
> > (<
> > > > v4),
> > > > >> > the join group request shall be downgraded to v3 without setting
> > the
> > > > >> member
> > > > >> > Id". It is probably simpler to just say that this feature is
> > enabled
> > > > if
> > > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > > >> MEMBER_NAME
> > > > >> > is configured with non-empty string.
> > > > >> > Yep, addressed this!
> > > > >> >
> > > > >> > 9) It is mentioned that broker may return
> > NO_STATIC_MEMBER_INFO_SET
> > > > >> error
> > > > >> > in OffsetCommitResponse for "commit requests under static
> > > membership".
> > > > >> Can
> > > > >> > you clarify how broker determines whether the commit request is
> > > under
> > > > >> > static membership?
> > > > >> >
> > > > >> > We have agreed that commit request shouldn't be affected by the
> > new
> > > > >> > membership, thus
> > > > >> > removing it here. Thanks for catching this!
> > > > >> >
> > > > >> > Let me know if you have further suggestions or concerns. Thank
> you
> > > for
> > > > >> > your valuable feedback
> > > > >> > to help me design the KIP better! (And I will try to address
> your
> > > > >> > feedbacks in next round Mayuresh ??)
> > > > >> >
> > > > >> > Best,
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hi Boyang,
> > > > >> >
> > > > >> > Thanks for updating the KIP. This is a step good direction for
> > > > stateful
> > > > >> > applications and also mirroring applications whose latency is
> > > affected
> > > > >> due
> > > > >> > to the rebalance issues that we have today.
> > > > >> >
> > > > >> > I had a few questions on the current version of the KIP :
> > > > >> > For the effectiveness of the KIP, consumer with member.name set
> > > will
> > > > >> *not
> > > > >> > send leave group request* when they go offline
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> > >
> > > > >> >
> > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > member.id
> > > > }
> > > > >> to
> > > > >> > track member uniqueness.
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > >
> > > > >> >
> > > > >> > Member.id must be set if the *member.name <
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=ckYE3gR46UxmbhDCqeZkfqfR%2F3sM60b8eZUFL0n8l%2F4%3D&amp;reserved=0
> > > > >> >
> > > > >> > *is already
> > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > >> >
> > > > >> > > How is this case handled on the client side? What is the
> > > application
> > > > >> that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > >
> > > > >> >
> > > > >> > Session timeout is the timeout we will trigger rebalance when a
> > > member
> > > > >> goes
> > > > >> > offline for too long (not sending heartbeat request). To make
> > static
> > > > >> > membership effective, we should increase the default max session
> > > > >> timeout to
> > > > >> > 30 min so that end user could config it freely.
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups. This is actually conflicting with
> the
> > > > >> design of
> > > > >> > static membership, because those temporarily unavailable members
> > > will
> > > > >> > potentially reattempt the join group and trigger extra
> rebalances.
> > > > >> > Internally we would optimize this logic by having rebalance
> > timeout
> > > > >> only in
> > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > >> non-responsive
> > > > >> > members immediately.
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > >
> > > > >> >
> > > > >> > Scale Up
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > *Discussion*
> > > > >> > Larger session timeouts causing latency rise for getting data
> for
> > > > >> un-owned
> > > > >> > topic partitions :
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > >
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Mayuresh
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > >> > konstantine@confluent.io> wrote:
> > > > >> >
> > > > >> > > Hi Boyang.
> > > > >> > >
> > > > >> > > Thanks for preparing this KIP! It is making good progress and
> > will
> > > > be
> > > > >> a
> > > > >> > > great improvement for stateful Kafka applications.
> > > > >> > >
> > > > >> > > Apologies for my late reply, I was away for a while. Lots of
> > great
> > > > >> > comments
> > > > >> > > so far, so I'll probably second most of them in what I suggest
> > > below
> > > > >> at
> > > > >> > > this point.
> > > > >> > >
> > > > >> > > When I first read the KIP, I wanted to start at the end with
> > > > something
> > > > >> > that
> > > > >> > > wasn't highlighted a lot. That was the topic related to
> handling
> > > > >> > duplicate
> > > > >> > > members. I see now that the initial suggestion of handling
> this
> > > > >> situation
> > > > >> > > during offset commit has been removed, and I agree with that.
> > > Issues
> > > > >> > > related to membership seem to be handled better when the
> member
> > > > joins
> > > > >> the
> > > > >> > > group rather than when it tries to commit offsets. This also
> > > > >> simplifies
> > > > >> > how
> > > > >> > > many request types need to change in order to incorporate the
> > new
> > > > >> member
> > > > >> > > name field.
> > > > >> > >
> > > > >> > > I also agree with what Jason and Guozhang have said regarding
> > > > >> timeouts.
> > > > >> > > Although semantically, it's easier to think of every operation
> > > > having
> > > > >> its
> > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > >> consolidation
> > > > >> > > seems preferable here. The definition of embedded protocols on
> > top
> > > > of
> > > > >> the
> > > > >> > > base group membership protocol for rebalancing gives enough
> > > > >> flexibility
> > > > >> > to
> > > > >> > > address such needs in each client component separately.
> > > > >> > >
> > > > >> > > Finally, some minor comments:
> > > > >> > > In a few places the new/proposed changes are referred to as
> > > > "current".
> > > > >> > > Which is a bit confusing considering that there is a protocol
> in
> > > > place
> > > > >> > > already, and by "current" someone might understand the
> existing
> > > one.
> > > > >> I'd
> > > > >> > > recommend using new/proposed or equivalent when referring to
> > > changes
> > > > >> > > introduced with KIP-345 and current/existing or equivalent
> when
> > > > >> referring
> > > > >> > > to existing behavior.
> > > > >> > >
> > > > >> > > There's the following sentence in the "Public Interfaces"
> > section:
> > > > >> > > "Since for many stateful consumer/stream applications, the
> state
> > > > >> > shuffling
> > > > >> > > is more painful than short time partial unavailability."
> > > > >> > > However, my understanding is that the changes proposed with
> > > KIP-345
> > > > >> will
> > > > >> > > not exploit any partial availability. A suggestion for dealing
> > > with
> > > > >> > > temporary imbalances has been made in "Incremental Cooperative
> > > > >> > Rebalancing"
> > > > >> > > which can work well with KIP-345, but here I don't see
> proposed
> > > > >> changes
> > > > >> > > that suggest that some resources (e.g. partitions) will keep
> > being
> > > > >> used
> > > > >> > > while others will not be utilized. Thus, you might want to
> > adjust
> > > > this
> > > > >> > > sentence. Correct me if I'm missing something related to that.
> > > > >> > >
> > > > >> > > In the rejected alternatives, under point 2) I read "we can
> copy
> > > the
> > > > >> > member
> > > > >> > > id to the config files". I believe it means to say "member
> name"
> > > > >> unless
> > > > >> > I'm
> > > > >> > > missing something about reusing member ids. Also below I read:
> > "By
> > > > >> > allowing
> > > > >> > > consumers to optionally specifying a member id" which probably
> > > > implies
> > > > >> > > "member name" again. In a sense this section highlights a
> > > potential
> > > > >> > > confusion between member name and member id. I wonder if we
> > could
> > > > >> come up
> > > > >> > > with a better term for the new field. StaticTag, StaticLabel,
> or
> > > > even
> > > > >> > > StaticName are some suggestions that could potentially help
> with
> > > > >> > confusion
> > > > >> > > between MemberId and MemberName and what corresponds to what.
> > But
> > > I
> > > > >> > > wouldn't like to disrupt the discussion with naming
> conventions
> > > too
> > > > >> much
> > > > >> > at
> > > > >> > > this point. I just mention it here as a thought.
> > > > >> > >
> > > > >> > > Looking forward to see the final details of this KIP. Great
> work
> > > so
> > > > >> far!
> > > > >> > >
> > > > >> > > Konstantine
> > > > >> > >
> > > > >> > >
> > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > bchen11@outlook.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Thanks Guozhang for the great summary here, and I have been
> > > > >> following
> > > > >> > up
> > > > >> > > > the action items here.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > timeout
> > > > and
> > > > >> > > > registration timeout. Great to see them being addressed in
> > > client
> > > > >> side!
> > > > >> > > >   2.  I double checked the design and I believe that it is
> ok
> > to
> > > > >> have
> > > > >> > > both
> > > > >> > > > static member and dynamic member co-exist in the same group.
> > So
> > > > the
> > > > >> > > upgrade
> > > > >> > > > shouldn't be destructive and we are removing the two
> > membership
> > > > >> > protocol
> > > > >> > > > switching APIs.
> > > > >> > > >   3.  I only have question about this one. I'm still reading
> > the
> > > > >> > > KafkaApis
> > > > >> > > > code here. Should I just use the same authorization logic
> for
> > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > > suggested,
> > > > >> > this
> > > > >> > > > feature could be better addressed in a separate KIP because
> it
> > > is
> > > > >> > pretty
> > > > >> > > > independent. I could start drafting the KIP once the current
> > > > >> proposal
> > > > >> > is
> > > > >> > > > approved.
> > > > >> > > >   5.  I believe that we don't need fencing in offset commit
> > > > request,
> > > > >> > > since
> > > > >> > > > duplicate member.name issue could be handled by join group
> > > > >> request. We
> > > > >> > > > shall reject join group with known member name but no member
> > id
> > > > >> (which
> > > > >> > > > means we already have an active member using this identity).
> > > > >> > > >   6.  I agree to remove that internal config once we move
> > > forward
> > > > >> with
> > > > >> > > > static membership. And I already removed the entire section
> > from
> > > > the
> > > > >> > KIP.
> > > > >> > > >
> > > > >> > > > Let me know if you have other concerns.
> > > > >> > > >
> > > > >> > > > Best,
> > > > >> > > > Boyang
> > > > >> > > > ________________________________
> > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > >> > > > To: dev
> > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > >> by
> > > > >> > > > specifying member id
> > > > >> > > >
> > > > >> > > > Hello Boyang,
> > > > >> > > >
> > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > appreciate
> > > > >> your
> > > > >> > > > patience answering to the feedbacks from the community. I'd
> > like
> > > > to
> > > > >> add
> > > > >> > > my
> > > > >> > > > 2cents here:
> > > > >> > > >
> > > > >> > > > 1. By introducing another two timeout configs,
> > > > registration_timeout
> > > > >> and
> > > > >> > > > expansion_timeout, we are effectively having four timeout
> > > configs:
> > > > >> > > session
> > > > >> > > > timeout, rebalance timeout (configured as "
> > max.poll.interval.ms
> > > "
> > > > on
> > > > >> > > client
> > > > >> > > > side), and these two. Interplaying these timeout configs can
> > be
> > > > >> quite
> > > > >> > > hard
> > > > >> > > > for users with such complexity, and hence I'm wondering if
> we
> > > can
> > > > >> > > simplify
> > > > >> > > > the situation with as less possible timeout configs as
> > possible.
> > > > >> Here
> > > > >> > is
> > > > >> > > a
> > > > >> > > > concrete suggestion I'd like propose:
> > > > >> > > >
> > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> addition
> > > to
> > > > >> the
> > > > >> > > > session_timeout for static members, we can just reuse the
> > > > >> > session_timeout
> > > > >> > > > and ask users to set it to a larger value when they are
> > > upgrading
> > > > a
> > > > >> > > dynamic
> > > > >> > > > client to a static client by setting the "member.name" at
> the
> > > > same
> > > > >> > time.
> > > > >> > > > By
> > > > >> > > > default, the broker-side min.session.timeout is 6 seconds
> and
> > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable to
> me
> > > (we
> > > > >> can
> > > > >> > of
> > > > >> > > > course modify this broker config to enlarge the valid
> interval
> > > if
> > > > we
> > > > >> > want
> > > > >> > > > in practice). And then we should also consider removing the
> > > > >> condition
> > > > >> > for
> > > > >> > > > marking a client as failed if the rebalance timeout has
> > reached
> > > > >> while
> > > > >> > the
> > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > session_timeout
> > > > >> > and
> > > > >> > > > rebalance_timeout are totally separated: the former is only
> > used
> > > > to
> > > > >> > > > determine if a consumer member of the group should be marked
> > as
> > > > >> failed
> > > > >> > > and
> > > > >> > > > kicked out of the group, and the latter is only used to
> > > determine
> > > > >> the
> > > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > > phase.
> > > > In
> > > > >> > > other
> > > > >> > > > words if a member did not send the JoinGroup in time of the
> > > > >> > > > rebalance_timeout, we still include it in the new generation
> > of
> > > > the
> > > > >> > group
> > > > >> > > > and use its old subscription info to send to leader for
> > > > assignment.
> > > > >> > Later
> > > > >> > > > if the member came back with HeartBeat request, we can still
> > > > follow
> > > > >> the
> > > > >> > > > normal path to bring it to the latest generation while
> > checking
> > > > that
> > > > >> > its
> > > > >> > > > sent JoinGroup request contains the same subscription info
> as
> > we
> > > > >> used
> > > > >> > to
> > > > >> > > > assign the partitions previously (which should be likely the
> > > case
> > > > in
> > > > >> > > > practice). In addition, we should let static members to not
> > send
> > > > the
> > > > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > > > static
> > > > >> > > member
> > > > >> > > > can only be leaving the group if its session has timed out,
> OR
> > > it
> > > > >> has
> > > > >> > > been
> > > > >> > > > indicated to not exist in the group any more (details
> below).
> > > > >> > > >
> > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > Cooperative
> > > > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> > > logic
> > > > >> at
> > > > >> > the
> > > > >> > > > application level, instead of at the protocol level. By
> doing
> > > this
> > > > >> we
> > > > >> > can
> > > > >> > > > also enable a few other optimizations, e.g. at the Streams
> > level
> > > > to
> > > > >> > first
> > > > >> > > > build up the state store as standby tasks and then trigger a
> > > > second
> > > > >> > > > rebalance to actually migrate the active tasks while keeping
> > the
> > > > >> actual
> > > > >> > > > rebalance latency and hence unavailability window to be
> small
> > (
> > > > >> > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=N8BTmhWAmDGJc9%2BQl6ulM9Qa5vzxIyXaGzCDILSIehs%3D&amp;reserved=0
> > > > >> > > ).
> > > > >> > > > I'd propose we align
> > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > >> > expansion_timeout
> > > > >> > > as
> > > > >> > > > part of the protocol layer, but only do that at the
> > > application's
> > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > > > still,
> > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > >> > > > <
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cbe90c4b8c80c424d34b808d655cddd4d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790740011984091&amp;sdata=VO0%2F5TczxUBuJkK7NurBwa1X0wIXwm0WHx4jrCzY0%2Fo%3D&amp;reserved=0
> > > > >> > > >*"
> > > > >> > > > though as part of this KIP
> > > > >> > > > since we have discussed about its limit and think it is
> > actually
> > > > >> not a
> > > > >> > > very
> > > > >> > > > good design and could be replaced with client-side logic
> > above.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> this
> > > KIP.
> > > > >> More
> > > > >> > > > specifically, let's say after we have upgraded broker
> version
> > to
> > > > be
> > > > >> > able
> > > > >> > > to
> > > > >> > > > recognize the new versions of JoinGroup request and the
> admin
> > > > >> requests,
> > > > >> > > how
> > > > >> > > > should we upgrade the clients and enable static groups? On
> top
> > > of
> > > > my
> > > > >> > head
> > > > >> > > > if we do a rolling bounce in which we set the member.name
> > > config
> > > > as
> > > > >> > well
> > > > >> > > > as
> > > > >> > > > optionally increase the session.timeout config when we
> bounce
> > > each
> > > > >> > > > instance, then during this rolling bounces we will have a
> > group
> > > > >> > contained
> > > > >> > > > with both dynamic members and static members. It means that
> we
> > > > >> should
> > > > >> > > have
> > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > JoinGroup
> > > > >> > > requests
> > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > member.id"
> > > > >> > > mapping
> > > > >> > > > will only be partial at this scenario. Also could you
> describe
> > > if
> > > > >> the
> > > > >> > > > upgrade to the first version that support this feature would
> > > ever
> > > > >> get
> > > > >> > any
> > > > >> > > > benefits, or only the future upgrade path for rolling
> bounces
> > > > could
> > > > >> get
> > > > >> > > > benefits out of this feature?
> > > > >> > > >
> > > > >> > > > If that's the case and we will do 1) as suggested above, do
> we
> > > > still
> > > > >> > need
> > > > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > > > >> requests
> > > > >> > any
> > > > >> > > > more? Seems it is not necessary any more as we will only
> have
> > > the
> > > > >> > notion
> > > > >> > > of
> > > > >> > > > "dynamic or static members" that can co-exist in a group
> while
> > > > >> there no
> > > > >> > > > notion of "dynamic or static groups", and hence these two
> > > requests
> > > > >> are
> > > > >> > > not
> > > > >> > > > needed anymore.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 3. We need to briefly talk about the implications for ACL as
> > we
> > > > >> > introduce
> > > > >> > > > new admin requests that are related to a specific group.id.
> > For
> > > > >> > example,
> > > > >> > > > we
> > > > >> > > > need to make sure that whoever created the group or joined
> the
> > > > group
> > > > >> > can
> > > > >> > > > actually send admin requests for the group, otherwise the
> > > > >> application
> > > > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > > > cluster
> > > > >> > every
> > > > >> > > > time they want to send any admin requests for their groups
> > which
> > > > >> would
> > > > >> > be
> > > > >> > > > an operational nightmare.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 4. I like Jason's suggestion of adding an optional field for
> > the
> > > > >> list
> > > > >> > of
> > > > >> > > > member names, and I'm wondering if that can be done as part
> of
> > > the
> > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > members,
> > > > we
> > > > >> > will
> > > > >> > > > enforce a rebalance immediately since it indicates that some
> > > > static
> > > > >> > > member
> > > > >> > > > will be officially kicked out of the group and some new
> static
> > > > >> members
> > > > >> > > may
> > > > >> > > > be added. So back to 1.a) above, a static member can only be
> > > > kicked
> > > > >> out
> > > > >> > > of
> > > > >> > > > the group if a) its session (arguably long period of time)
> has
> > > > timed
> > > > >> > out,
> > > > >> > > > and b) this admin request explicitly state that it is no
> > longer
> > > > >> part of
> > > > >> > > the
> > > > >> > > > group. As for execution I'm fine with keeping it as a future
> > > work
> > > > of
> > > > >> > this
> > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > >> > > >
> > > > >> > > > Following are minor comments:
> > > > >> > > >
> > > > >> > > > 5. I'm not sure if we need to include "member.name" as part
> > of
> > > > the
> > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > memberId
> > > > >> plus
> > > > >> > > the
> > > > >> > > > generation number should be sufficient for fencing even with
> > > > static
> > > > >> > > > members.
> > > > >> > > >
> > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid
> of
> > > the
> > > > "
> > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > Guozhang
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> lindong28@gmail.com
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Hey Boyang,
> > > > >> > > > >
> > > > >> > > > > Thanks for the proposal! This is very useful. I have some
> > > > comments
> > > > >> > > below:
> > > > >> > > > >
> > > > >> > > > > 1) The motivation currently explicitly states that the
> goal
> > is
> > > > to
> > > > >> > > improve
> > > > >> > > > > performance for heavy state application. It seems that the
> > > > >> motivation
> > > > >> > > can
> > > > >> > > > > be stronger with the following use-case. Currently for
> > > > MirrorMaker
> > > > >> > > > cluster
> > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> > time
> > > to
> > > > >> > > rolling
> > > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > > process
> > > > >> > restart
> > > > >> > > > > will trigger a rebalance which currently pause the
> > consumption
> > > > of
> > > > >> the
> > > > >> > > all
> > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > stated
> > > in
> > > > >> this
> > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > specified
> > > > >> > > timeout
> > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > rebalance
> > > > >> > (e.g.
> > > > >> > > > for
> > > > >> > > > > leader restart) for the entire rolling bounce, which will
> > > > >> > significantly
> > > > >> > > > > improves the availability of the MirrorMaker pipeline. In
> my
> > > > >> opinion,
> > > > >> > > the
> > > > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance
> if
> > > the
> > > > >> > > consumer
> > > > >> > > > > process can be restarted within soon, which helps
> > performance
> > > > >> even if
> > > > >> > > > > overhead of state shuffling for a given process is small.
> > > > >> > > > >
> > > > >> > > > > 2) In order to simplify the KIP reading, can you follow
> the
> > > > >> writeup
> > > > >> > > style
> > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> > such
> > > as
> > > > >> new
> > > > >> > > > > configs (e.g. registration timeout), new request/response,
> > new
> > > > >> > > > AdminClient
> > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > Currently
> > > > >> some
> > > > >> > > of
> > > > >> > > > > these are specified in the Proposed Change section which
> > makes
> > > > it
> > > > >> a
> > > > >> > bit
> > > > >> > > > > inconvenient to understand the new interface that will be
> > > > exposed
> > > > >> to
> > > > >> > > > user.
> > > > >> > > > > Explanation of the current two-phase rebalance protocol
> > > probably
> > > > >> can
> > > > >> > be
> > > > >> > > > > moved out of public interface section.
> > > > >> > > > >
> > > > >> > > > > 3) There are currently two version of JoinGroupRequest in
> > the
> > > > KIP
> > > > >> and
> > > > >> > > > only
> > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > >> > > > >
> > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > > > >> rebalance
> > > > >> > > could
> > > > >> > > > > be helpful here, but we will make a call once we finished
> > the
> > > > >> major
> > > > >> > > > > implementation". So this seems to be still an open
> question
> > in
> > > > the
> > > > >> > > > current
> > > > >> > > > > design. We probably want to agree on this before voting
> for
> > > the
> > > > >> KIP.
> > > > >> > > > >
> > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > consumer.
> > > > Can
> > > > >> > you
> > > > >> > > > > specify the name of the config key and the default config
> > > value?
> > > > >> > > Possible
> > > > >> > > > > default values include empty string or null (similar to
> > > > >> > transaction.id
> > > > >> > > > in
> > > > >> > > > > producer config).
> > > > >> > > > >
> > > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > > persist
> > > > >> > member
> > > > >> > > > > name map, currently if consumer coordinator broker goes
> > > offline,
> > > > >> > > > rebalance
> > > > >> > > > > is triggered and consumers will try connect to the new
> > > > >> coordinator.
> > > > >> > If
> > > > >> > > > > these consumers can connect to the new coordinator within
> > > > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> > > that
> > > > >> > broker
> > > > >> > > > can
> > > > >> > > > > use a deterministic algorithm to determine the partition
> ->
> > > > >> > member_name
> > > > >> > > > > mapping, each consumer should get assigned the same set of
> > > > >> partitions
> > > > >> > > > > without requiring state shuffling. So it is not clear
> > whether
> > > we
> > > > >> > have a
> > > > >> > > > > strong use-case for this new logic. Can you help clarify
> > what
> > > is
> > > > >> the
> > > > >> > > > > benefit of using topic "static_member_map" to persist
> member
> > > > name
> > > > >> > map?
> > > > >> > > > >
> > > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > > config,
> > > > >> it is
> > > > >> > > > > mentioned that "we are using expansion timeout to replace
> > > > >> rebalance
> > > > >> > > > > timeout, which is configured by max.poll.intervals from
> > client
> > > > >> side,
> > > > >> > > and
> > > > >> > > > > using registration timeout to replace session timeout".
> > > > Currently
> > > > >> the
> > > > >> > > > > default max.poll.interval.ms is configured to be 5
> minutes
> > > and
> > > > >> there
> > > > >> > > > will
> > > > >> > > > > be only one rebalance if all new consumers can join
> within 5
> > > > >> minutes.
> > > > >> > > So
> > > > >> > > > it
> > > > >> > > > > is not clear whether we have a strong use-case for this
> new
> > > > >> config.
> > > > >> > Can
> > > > >> > > > you
> > > > >> > > > > explain what is the benefit of introducing this new
> config?
> > > > >> > > > >
> > > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > > version
> > > > >> of
> > > > >> > > > > protocol, we will also increase the join group request
> > version
> > > > to
> > > > >> v4
> > > > >> > > when
> > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> > > latest
> > > > >> (<
> > > > >> > > v4),
> > > > >> > > > > the join group request shall be downgraded to v3 without
> > > setting
> > > > >> the
> > > > >> > > > member
> > > > >> > > > > Id". It is probably simpler to just say that this feature
> is
> > > > >> enabled
> > > > >> > if
> > > > >> > > > > JoinGroupRequest V4 is supported on both client and broker
> > and
> > > > >> > > > MEMBER_NAME
> > > > >> > > > > is configured with non-empty string.
> > > > >> > > > >
> > > > >> > > > > 9) It is mentioned that broker may return
> > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > >> > > error
> > > > >> > > > > in OffsetCommitResponse for "commit requests under static
> > > > >> > membership".
> > > > >> > > > Can
> > > > >> > > > > you clarify how broker determines whether the commit
> request
> > > is
> > > > >> under
> > > > >> > > > > static membership?
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > > Dong
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > --
> > > > >> > > > -- Guozhang
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > --
> > > > >> > -Regards,
> > > > >> > Mayuresh R. Gharat
> > > > >> > (862) 250-7125
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -- Guozhang
>


--
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Guozhang for the new proposal here!

So I'd like to propose a slightly modified version of LeaveGroupRequest:
instead of letting the static member consumer client themselves to send the
request (which means we still need to have some hidden configs to turn it
off like we did today), how about just letting any other client to send
this request since the LeaveGroupRequest only requires group.id and
member.id? So back to your operational scenarios, if some static member has
been found crashed and it is not likely to comeback, or we simply want to
shrink the size of the group by shutting down some static members, we can
use an admin client to send the LeaveGroupRequest after the instance has
been completely shutdown or crashed to kick them out of the group and also
triggers the rebalance.

One issue though, is that users may not know the member id required in the
LeaveGroupRequest. To work around it we can add the `group.instance.id`
along with the member id as well and then allow member id null-able. The
coordinator logic would then be modified as 1) if member.id is specified,
ignore instance.id and always use member.id to find the member to kick out,
2) otherwise, try with the instance.id to find the corresponding member.id
and kick it out, 3) if none is found, reject with an error code.

So in sum the alternative changes are:

a) Modify LeaveGroupRequest to add group.instance.id
b) Modify coordinator logic to handle such request on the broker side.
c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
instanceId)" which will be translated as a LeaveGroupRequest.
d) [Optional] we can even batch the request by allowing
"removeMemberFromGroup(groupId, list[instanceId])" and then make `member.id`
and `instance.id` field of LeaveGroupRequest to be an array instead of a
single entry.
e) We can also remove the admin ConsumerRebalanceRequest as well for
simplicity (why not? paranoid of having as less request protocols as
possible :), as it is not needed anymore with the above proposal.
I agree that reusing LeaveGroupRequest is actually a good idea: we only need to iterate
over an existing request format. Also I found that we haven't discussed how we want to enable
this feature on Streaming applications, which is different from common consumer application in that
Stream app uses stream thread as individual consumer.
For example if user specifies the client id, the stream consumer client id will be like:
User client id + "-StreamThread-" + thread id + "-consumer"

So I'm thinking we should do sth similar for defining group.instance.id on Stream. We shall define another
config called `stream.instance.id` which would be used as prefix, and for each thread consumer the formula
will look like:
`group.instance.id` = `stream.instance.id` + "-" + thread id + "-consumer"

And for the ease of use, the interface of leave group request could include `group.instance.id.prefix` instead of
`group.instance.id` so that we could batch remove consumers relating to a single stream instance. This is more intuitive
and flexible since specifying names of 16~32 * n (n = number of stream instances to shut down) consumers is not an easy
job without client management tooling.

How does this workaround sound?

Boyang
________________________________
From: Guozhang Wang <wa...@gmail.com>
Sent: Thursday, November 29, 2018 2:38 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

I was thinking that with the optional static members in the admin
ConsumerRebalanceRequest it should be sufficient to kick out the static
member before their session timeout (arguably long in practice) have not
reached. But now I see your concern is that in some situations the admin
operators may not even know the full list of static members, but ONLY know
which static member has failed and hence would like to kick out of the
group.

So I'd like to propose a slightly modified version of LeaveGroupRequest:
instead of letting the static member consumer client themselves to send the
request (which means we still need to have some hidden configs to turn it
off like we did today), how about just letting any other client to send
this request since the LeaveGroupRequest only requires group.id and
member.id? So back to your operational scenarios, if some static member has
been found crashed and it is not likely to comeback, or we simply want to
shrink the size of the group by shutting down some static members, we can
use an admin client to send the LeaveGroupRequest after the instance has
been completely shutdown or crashed to kick them out of the group and also
triggers the rebalance.

One issue though, is that users may not know the member id required in the
LeaveGroupRequest. To work around it we can add the `group.instance.id`
along with the member id as well and then allow member id null-able. The
coordinator logic would then be modified as 1) if member.id is specified,
ignore instance.id and always use member.id to find the member to kick out,
2) otherwise, try with the instance.id to find the corresponding member.id
and kick it out, 3) if none is found, reject with an error code.

So in sum the alternative changes are:

a) Modify LeaveGroupRequest to add group.instance.id
b) Modify coordinator logic to handle such request on the broker side.
c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
instanceId)" which will be translated as a LeaveGroupRequest.
d) [Optional] we can even batch the request by allowing
"removeMemberFromGroup(groupId, list[instanceId])" and then make `member.id`
and `instance.id` field of LeaveGroupRequest to be an array instead of a
single entry.
e) We can also remove the admin ConsumerRebalanceRequest as well for
simplicity (why not? paranoid of having as less request protocols as
possible :), as it is not needed anymore with the above proposal.


WDYT?


Guozhang

On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Guozhang and Mayuresh for the follow up! Answers are listed below.
>
>
> >  5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
>
> Great question. Let's use c1~c4 example here:
>
>   1.  Consumer c1, c2, c3, c4 in stable state
>   2.  c4 goes down and we detect this issue before session timeout through
> client monitoring. Initiate a ConsumerRebalanceRequest.
>   3.  A rebalance will be kicking off, and after rebalance timeout we
> shall keep the same assignment for c1~4, if the session timeout for c4
> hasn't reached
>   4.  Group back to stable with c1~4 (although c4 is actually offline)
>   5.  c4 session timeout finally reached: another rebalance triggered.
>
> For step 3, if session timeout triggered within rebalance timeout, only
> c1~3 will be participating in the rebalance. This is what we mean by saying
> "rebalance
> timeout shall not remove current members, only session timeout will do."
> As you could see this is not an ideal scenario: we trigger extra rebalance
> at step 5. In my reply to Guozhang I'm asking whether we should still use
> LeaveGroupRequest for static members to send a signal to broker saying "I'm
> currently offline", and when we send ConsumerRebalanceRequest to broker, we
> will actually kick off c4 because it says it's offline already, saving one
> or multiple additional rebalances later. This way the
> ConsumerRebalanceRequest will be more effective in making correct judgement
> on the group status since we have more feedback from client side.
>
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> Previous proposal should be suffice to answer this question 😊
>
> - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763089623&amp;sdata=l%2FDAHyVIG1EYY4Vsqn6jzybyqca67mZlmgZOh1i%2F8mI%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> Thanks for catching it! And yes, we shall use one extra round-trip between
> consumer
> and broker to inform the new member id allocation.
>
> Next is the replies to Guozhang's comment:
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
> I'm also in favor of storing all the in-group static members. In fact we
> could reuse
> the static membership mapping to store this information. Do you think
> that we should let static member send leave group request to indicate
> their status of "leaving",
> and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> suggesting we should
> remove those members when kicking off rebalance since we are shutting them
> down already.
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
> Sure we have enough votes for ids 😊I will finalize the name to `
> group.instance.id`, does that
> sound good?
>
> Best,
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Wednesday, November 28, 2018 4:51 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Regarding Jason's question and Boyang's responses:
>
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
>
>
> Guozhang
>
>
> On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Thanks for the replies. Please find the follow up queries below.
> >
> >     5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
> > The plan is to disable member kick out when rebalance.timeout is reached,
> > so basically we are not "waiting" any
> > join group request from existing members; we shall just rebalance base on
> > what we currently have within the group
> > metadata. Lagging consumer will trigger rebalance later if session
> timeout
> > > rebalance timeout.
> >
> > >
> > Just wanted to understand this better. Lets take an example, say we have
> a
> > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > Everything is running fine and suddenly C4 host has issues and it goes
> > > down. Now we notice that we can still operate with c1, c2, c3 and don't
> > > want to wait for
> > > c4 to come back up. We use the admin api
> > > "invokeConsumerRebalance("GroupA")".
> > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > > group again (in there heartBeatResponse) as first step of rebalance.
> > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> > > delayed. At this stage, if we are not "waiting" on any join group
> > request,
> > > few things can happen :
> > >
> > >    - c4's partitions are distributed only among c1,c2. c3 maintains its
> > >    original assignment. c1, c2 will start processing the newly assigned
> > >    partitions.
> > >
> > > OR
> > >
> > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
> > >    processing the newly assigned partitions. c3 gets to know about the
> > newly
> > >    assigned partitions later when it sends the JoinGroupRequest (which
> > was
> > >    delayed).
> > >
> > > OR
> > >
> > >    - Will the rebalance do a complete reassignment, where c1, c2, c3
> have
> > >    to give up there partitions and all the partitions belonging to c1,
> > c2, c3,
> > >    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
> > >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> > revoke
> > >    there partitions and rejoin the group.
> > >
> > > This is as per my understanding of how the KIP would work without
> > changing
> > > the underlying group coordination workflow. Please correct me if I
> > > misunderstood something here.
> > >
> >
> >
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> >
> > - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763089623&amp;sdata=l%2FDAHyVIG1EYY4Vsqn6jzybyqca67mZlmgZOh1i%2F8mI%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> both
> > > in this reply.
> > >
> > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > >
> > > No, we shall only rely on broker to allocate member.id for the
> consumer
> > > instances. FYI, I already
> > >
> > > started the discussion thread for KIP-394 😊
> > >
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > >
> > > Yes, we could only use unknown member id. Actually this part of the
> > > proposal is outdated,
> > >
> > > let me do another audit of the whole doc. Basically, it is currently
> > > impossible to send `member.id`
> > >
> > > when consumer restarted. Sorry for the confusions!
> > >
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763089623&amp;sdata=z2dfArhlUOFw4q2mYUwx7SEdvkNz7lO%2FPyHOjro7Dco%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763089623&amp;sdata=CcpbLAuljqPTKM0Ql22J4PMOd%2BmP0GTQvHmWsZr3VuI%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=LTlh0cgGjYQQs4jbgXWK34A4e6jNHXxxJwqx%2F19XfRU%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > >
> > > I would take Jason's advice that each time we have unknown member
> joining
> > > the group, the broker will
> > >
> > > always assign a new and unique id to track its identity. In this way,
> > > consumer with duplicate member name
> > >
> > > will be fenced.
> > >
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > >
> > > Like explained above, we shall not materialize the member.id. Instead
> we
> > > need to rely on broker to allocate
> > >
> > > a unique id for consumer just like what we have now.
> > >
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > >
> > > The plan is to disable member kick out when rebalance.timeout is
> reached,
> > > so basically we are not "waiting" any
> > >
> > > join group request from existing members; we shall just rebalance base
> on
> > > what we currently have within the group
> > >
> > > metadata. Lagging consumer will trigger rebalance later if session
> > timeout
> > > > rebalance timeout.
> > >
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > >
> > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > >
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > >
> > > Nah, I'm gonna remove it because it seems causing more confusion than
> > > making my assumption clear, which is
> > >
> > > "there could be other failure cases that I can't enumerate now" 😊
> > >
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > >
> > > Great suggestion! Added a simple rollback plan.
> > >
> > >
> > > Next is answering Jason's suggestions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > Yep, I like this idea! Keep giving out refresh member.id when facing
> > > anonymous request will definitely
> > >
> > > prevent processing bug due to duplicate consumers, however I don't
> think
> > I
> > > fully understand the 3rd paragraph where
> > >
> > > you mentioned  "There needs to be an expectation for static members
> that
> > > the member.id of a static member will
> > >
> > > not be changed except when a new member with the same member.name
> joins
> > > the group. "  How do you plan
> > > to know whether this member is new member or old member? I feel even
> with
> > > zombie consumer takes the ownership,
> > > it should be detected very quickly (as MISMATCH_ID exception trigger
> > > original consumer instance dies)
> > > and end user will start to fix it right away. Is there any similar
> logic
> > > we applied in fencing duplicate `transaction.id`?
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > This API is needed when we need to immediately trigger rebalance
> instead
> > > of waiting session timeout
> > >
> > > or rebalance timeout (Emergent scale up/down). It is very necessary to
> > > have it for
> > >
> > > management purpose because user could choose when to trigger rebalance
> > > pretty freely,
> > >
> > > gaining more client side control.
> > >
> > > In the meanwhile I see your point that we need to actually have the
> > > ability to kick out members that we plan
> > >
> > > to scale down fast (as rebalance timeout no longer kicks any offline
> > > member out of the group), I will think of adding an optional
> > >
> > > list of members that are ready to be removed.
> > >
> > > Another idea is to let static member send `LeaveGroupRequest` when they
> > > are going offline (either scale down or bouncing),
> > >
> > > and broker will cache this information as "OfflineMembers" without
> > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> will
> > >
> > > kick the static members that are currently offline and trigger
> rebalance
> > > immediately. How does this plan sound?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> > > the namespace idea, and
> > >
> > > since we already use `member.name` in a lot of context, I decide to
> > > rename the config to `group.member.name`
> > >
> > > which should be sufficient for solving all the concerns we have now.
> > > Sounds good?
> > >
> > >
> > > Thank you for your great suggestions! Let me know if my reply makes
> sense
> > > her.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Jason Gustafson <ja...@confluent.io>
> > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the updates. Looks like we're headed in the right direction
> > and
> > > clearly the interest that this KIP is receiving shows how strong the
> > > motivation is!
> > >
> > > I have a few questions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks a lot for replying to all the queries and discussions here, so
> > > > patiently.
> > > > Really appreciate it.
> > > >
> > > > Had a few questions and suggestions after rereading the current
> version
> > > of
> > > > the KIP :
> > > >
> > > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=LTlh0cgGjYQQs4jbgXWK34A4e6jNHXxxJwqx%2F19XfRU%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=hiLbw0%2FH5Cs4An2cGra27IJvKZHEr9lu2BePYh0jud4%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=LTlh0cgGjYQQs4jbgXWK34A4e6jNHXxxJwqx%2F19XfRU%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > gharatmayuresh15@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > > >
> > > > >> Hey Dong, thanks for the follow-up here!
> > > > >>
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >> Great question. I feel `member.name` is enough to explain itself,
> > it
> > > > >> seems not very
> > > > >> helpful to make the config name longer. Comparing `name` with `id`
> > > gives
> > > > >> user the
> > > > >> impression that they have the control over it with customized rule
> > > than
> > > > >> library decided.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >> We are not changing the default session timeout value. We are just
> > > > >> changing the
> > > > >> cap we are enforcing on the session timeout max value. So this
> > change
> > > is
> > > > >> not affecting
> > > > >> what kind of membership end user is using, and loosing the cap is
> > > giving
> > > > >> end user
> > > > >> more flexibility on trade-off between liveness and stability.
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> because
> > > with
> > > > >> the KIP-394<
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=fDSwi4Sl239xrDBRs%2BKUf79hlGqJIMfijuEkHY5sKwE%3D&amp;reserved=0
> > > > >> >
> > > > >> we will automatically fence all join requests with
> > UNKNOWN_MEMBER_ID.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >> Same answer as question 3). This part of the logic shall be
> removed
> > > from
> > > > >> the proposal.
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >> Sounds good! Added both methods.
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=nRirA2TsjrZ9GdFRYYgsaEfzNuH%2FeMS8VLyjeKgdHsE%3D&amp;reserved=0
> > > > >> .
> > > > >> Added the script.
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >> Sounds good. Right now I feel for both static and dynamic
> membership
> > > it
> > > > is
> > > > >> more manageable to introduce the consumer rebalance method through
> > > admin
> > > > >> client API.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Sure, I will add more details to the API.
> > > > >>
> > > > >>
> > > > >> Thanks again for the helpful suggestions!
> > > > >>
> > > > >>
> > > > >> Best,
> > > > >> Boyang
> > > > >>
> > > > >> ________________________________
> > > > >> From: Dong Lin <li...@gmail.com>
> > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > >> To: dev
> > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > > >> specifying member id
> > > > >>
> > > > >> Hey Boyang,
> > > > >>
> > > > >> Thanks for the update! Here are some followup comments:
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=nRirA2TsjrZ9GdFRYYgsaEfzNuH%2FeMS8VLyjeKgdHsE%3D&amp;reserved=0
> > > > >> .
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Thanks,
> > > > >> Dong
> > > > >>
> > > > >>
> > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > > > wrote:
> > > > >>
> > > > >> > Hey Mayuresh,
> > > > >> >
> > > > >> >
> > > > >> > thanks for your feedbacks! I will try do another checklist here.
> > > > >> >
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> >
> > > > >> > Yep it's true, we will prevent client from sending leave group
> > > request
> > > > >> > when they are set with `member.name`.
> > > > >> >
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > Good question, we will only kick off member due to session
> timeout
> > > > >> within
> > > > >> > static membership. Let me update the KIP to clearly assert that.
> > > > >> >
> > > > >> > > How is this case (missing member id) handled on the client
> side?
> > > > What
> > > > >> is
> > > > >> > the application that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > I have extended the two exceptions within join group response
> V4.
> > > > >> > Basically I define both corresponding actions to be immediate
> > > failing
> > > > >> > client application, because so far it is unknown what kind of
> > client
> > > > >> issue
> > > > >> > could trigger them. After the first version, we will keep
> enhance
> > > the
> > > > >> error
> > > > >> > handling logic!
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > The goal of extending session timeout cap is for users with good
> > > > client
> > > > >> > side monitoring tools that could auto-heal the dead consumers
> very
> > > > >> fast. So
> > > > >> > it is optional (and personal) to extend session timeout to a
> > > > reasonable
> > > > >> > number with different client scenarios.
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > functionality
> > > > >> > to only use as a timer to
> > > > >> > end prepare rebalance stage. This way, late joining static
> members
> > > > will
> > > > >> > not trigger further rebalance
> > > > >> > as long as they are within session timeout. I added your
> highlight
> > > to
> > > > >> the
> > > > >> > KIP!
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> because
> > we
> > > > >> > believe client side could
> > > > >> > better handle this logic.
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > Great idea! In fact on top of static membership we could later
> > > > introduce
> > > > >> > APIs to set hard-coded
> > > > >> > client ids to the group and replace the dead host, or as you
> > > proposed
> > > > to
> > > > >> > define spare host as
> > > > >> > what I understood as hot backup. I will put both Jason and your
> > > > >> > suggestions into a separate section
> > > > >> > called "Future works". Note that this spare host idea may be
> also
> > > > >> solvable
> > > > >> > through rebalance protocol
> > > > >> > IMO.
> > > > >> >
> > > > >> > Thank you again for the great feedback!
> > > > >> >
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> > email
> > > > on
> > > > >> my
> > > > >> > thread, so I will just do a checklist here!
> > > > >> >
> > > > >> >
> > > > >> > 1) The motivation currently explicitly states that the goal is
> to
> > > > >> improve
> > > > >> >
> > > > >> > performance for heavy state application. It seems that the
> > > motivation
> > > > >> can
> > > > >> >
> > > > >> > be stronger with the following use-case. Currently for
> MirrorMaker
> > > > >> cluster
> > > > >> >
> > > > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > > rolling
> > > > >> >
> > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > > > restart
> > > > >> >
> > > > >> > will trigger a rebalance which currently pause the consumption
> of
> > > the
> > > > >> all
> > > > >> >
> > > > >> > partitions of the MirrorMaker cluster. With the change stated in
> > > this
> > > > >> >
> > > > >> > patch, as long as a MirrorMaker can restart within the specified
> > > > timeout
> > > > >> >
> > > > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> > > (e.g.
> > > > >> for
> > > > >> >
> > > > >> > leader restart) for the entire rolling bounce, which will
> > > > significantly
> > > > >> >
> > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > opinion,
> > > > >> the
> > > > >> >
> > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > > >> consumer
> > > > >> >
> > > > >> > process can be restarted within soon, which helps performance
> even
> > > if
> > > > >> >
> > > > >> > overhead of state shuffling for a given process is small.
> > > > >> >
> > > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > > making
> > > > >> the
> > > > >> > motivation more solid!
> > > > >> >
> > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > writeup
> > > > >> style
> > > > >> > of other KIP (e.g. KIP-98) and list the interface change such as
> > new
> > > > >> > configs (e.g. registration timeout), new request/response, new
> > > > >> AdminClient
> > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> > > some
> > > > of
> > > > >> > these are specified in the Proposed Change section which makes
> it
> > a
> > > > bit
> > > > >> > inconvenient to understand the new interface that will be
> exposed
> > to
> > > > >> user.
> > > > >> > Explanation of the current two-phase rebalance protocol probably
> > can
> > > > be
> > > > >> > moved out of public interface section.
> > > > >> > This is a great suggestion! I just consolidated all the public
> API
> > > > >> > changes, and the whole KIP
> > > > >> > looks much more organized!
> > > > >> >
> > > > >> > 3) There are currently two version of JoinGroupRequest in the
> KIP
> > > and
> > > > >> only
> > > > >> > one of them has field memberId. This seems confusing.
> > > > >> > Yep, I already found this issue and fixed it.
> > > > >> >
> > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > rebalance
> > > > >> could
> > > > >> > be helpful here, but we will make a call once we finished the
> > major
> > > > >> > implementation". So this seems to be still an open question in
> the
> > > > >> current
> > > > >> > design. We probably want to agree on this before voting for the
> > KIP.
> > > > >> > We have finalized the idea that this API is needed.
> > > > >> >
> > > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> Can
> > > you
> > > > >> > specify the name of the config key and the default config value?
> > > > >> Possible
> > > > >> > default values include empty string or null (similar to
> > > > transaction.id<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=vUh2BXs7INb8zyKjPSfnZFFySLTzBB8UgC8eOBdkLHY%3D&amp;reserved=0
> > > > >> >
> > > > >> > in
> > > > >> > producer config).
> > > > >> > I have defined the `member.name` in "New configuration"
> section.
> > > > >> >
> > > > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > > > member
> > > > >> > name map, currently if consumer coordinator broker goes offline,
> > > > >> rebalance
> > > > >> > is triggered and consumers will try connect to the new
> > coordinator.
> > > If
> > > > >> > these consumers can connect to the new coordinator within
> > > > >> > max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=6a5sqZzDOOXhRKTCSmlbs0qZrF3aKcDo7tHxJDPYlU4%3D&amp;reserved=0
> > > > >> >
> > > > >> > which by default is 5 minutes, given that broker can
> > > > >> > use a deterministic algorithm to determine the partition ->
> > > > member_name
> > > > >> > mapping, each consumer should get assigned the same set of
> > > partitions
> > > > >> > without requiring state shuffling. So it is not clear whether we
> > > have
> > > > a
> > > > >> > strong use-case for this new logic. Can you help clarify what is
> > the
> > > > >> > benefit of using topic "static_member_map" to persist member
> name
> > > map?
> > > > >> > I have discussed with Guozhang offline, and I believe reusing
> the
> > > > >> current
> > > > >> > `_consumer_offsets`
> > > > >> > topic is a better and unified solution.
> > > > >> >
> > > > >> > 7) Regarding the introduction of the expensionTimeoutMs config,
> it
> > > is
> > > > >> > mentioned that "we are using expansion timeout to replace
> > rebalance
> > > > >> > timeout, which is configured by max.poll.intervals from client
> > side,
> > > > and
> > > > >> > using registration timeout to replace session timeout".
> Currently
> > > the
> > > > >> > default max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=6a5sqZzDOOXhRKTCSmlbs0qZrF3aKcDo7tHxJDPYlU4%3D&amp;reserved=0
> > > > >> >
> > > > >> > is configured to be 5 minutes and there will
> > > > >> > be only one rebalance if all new consumers can join within 5
> > > minutes.
> > > > >> So it
> > > > >> > is not clear whether we have a strong use-case for this new
> > config.
> > > > Can
> > > > >> you
> > > > >> > explain what is the benefit of introducing this new config?
> > > > >> > Previously our goal is to use expansion timeout as a workaround
> > for
> > > > >> > triggering multiple
> > > > >> > rebalances when scaling up members are not joining at the same
> > time.
> > > > It
> > > > >> is
> > > > >> > decided to
> > > > >> > be addressed by client side protocol change, so we will not
> > > introduce
> > > > >> > expansion timeout.
> > > > >> >
> > > > >> > 8) It is mentioned that "To distinguish between previous version
> > of
> > > > >> > protocol, we will also increase the join group request version
> to
> > v4
> > > > >> when
> > > > >> > MEMBER_NAME is set" and "If the broker version is not the latest
> > (<
> > > > v4),
> > > > >> > the join group request shall be downgraded to v3 without setting
> > the
> > > > >> member
> > > > >> > Id". It is probably simpler to just say that this feature is
> > enabled
> > > > if
> > > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > > >> MEMBER_NAME
> > > > >> > is configured with non-empty string.
> > > > >> > Yep, addressed this!
> > > > >> >
> > > > >> > 9) It is mentioned that broker may return
> > NO_STATIC_MEMBER_INFO_SET
> > > > >> error
> > > > >> > in OffsetCommitResponse for "commit requests under static
> > > membership".
> > > > >> Can
> > > > >> > you clarify how broker determines whether the commit request is
> > > under
> > > > >> > static membership?
> > > > >> >
> > > > >> > We have agreed that commit request shouldn't be affected by the
> > new
> > > > >> > membership, thus
> > > > >> > removing it here. Thanks for catching this!
> > > > >> >
> > > > >> > Let me know if you have further suggestions or concerns. Thank
> you
> > > for
> > > > >> > your valuable feedback
> > > > >> > to help me design the KIP better! (And I will try to address
> your
> > > > >> > feedbacks in next round Mayuresh ??)
> > > > >> >
> > > > >> > Best,
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hi Boyang,
> > > > >> >
> > > > >> > Thanks for updating the KIP. This is a step good direction for
> > > > stateful
> > > > >> > applications and also mirroring applications whose latency is
> > > affected
> > > > >> due
> > > > >> > to the rebalance issues that we have today.
> > > > >> >
> > > > >> > I had a few questions on the current version of the KIP :
> > > > >> > For the effectiveness of the KIP, consumer with member.name set
> > > will
> > > > >> *not
> > > > >> > send leave group request* when they go offline
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> > >
> > > > >> >
> > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > member.id
> > > > }
> > > > >> to
> > > > >> > track member uniqueness.
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > >
> > > > >> >
> > > > >> > Member.id must be set if the *member.name <
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=hiLbw0%2FH5Cs4An2cGra27IJvKZHEr9lu2BePYh0jud4%3D&amp;reserved=0
> > > > >> >
> > > > >> > *is already
> > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > >> >
> > > > >> > > How is this case handled on the client side? What is the
> > > application
> > > > >> that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > >
> > > > >> >
> > > > >> > Session timeout is the timeout we will trigger rebalance when a
> > > member
> > > > >> goes
> > > > >> > offline for too long (not sending heartbeat request). To make
> > static
> > > > >> > membership effective, we should increase the default max session
> > > > >> timeout to
> > > > >> > 30 min so that end user could config it freely.
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups. This is actually conflicting with
> the
> > > > >> design of
> > > > >> > static membership, because those temporarily unavailable members
> > > will
> > > > >> > potentially reattempt the join group and trigger extra
> rebalances.
> > > > >> > Internally we would optimize this logic by having rebalance
> > timeout
> > > > >> only in
> > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > >> non-responsive
> > > > >> > members immediately.
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > >
> > > > >> >
> > > > >> > Scale Up
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > *Discussion*
> > > > >> > Larger session timeouts causing latency rise for getting data
> for
> > > > >> un-owned
> > > > >> > topic partitions :
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > >
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Mayuresh
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > >> > konstantine@confluent.io> wrote:
> > > > >> >
> > > > >> > > Hi Boyang.
> > > > >> > >
> > > > >> > > Thanks for preparing this KIP! It is making good progress and
> > will
> > > > be
> > > > >> a
> > > > >> > > great improvement for stateful Kafka applications.
> > > > >> > >
> > > > >> > > Apologies for my late reply, I was away for a while. Lots of
> > great
> > > > >> > comments
> > > > >> > > so far, so I'll probably second most of them in what I suggest
> > > below
> > > > >> at
> > > > >> > > this point.
> > > > >> > >
> > > > >> > > When I first read the KIP, I wanted to start at the end with
> > > > something
> > > > >> > that
> > > > >> > > wasn't highlighted a lot. That was the topic related to
> handling
> > > > >> > duplicate
> > > > >> > > members. I see now that the initial suggestion of handling
> this
> > > > >> situation
> > > > >> > > during offset commit has been removed, and I agree with that.
> > > Issues
> > > > >> > > related to membership seem to be handled better when the
> member
> > > > joins
> > > > >> the
> > > > >> > > group rather than when it tries to commit offsets. This also
> > > > >> simplifies
> > > > >> > how
> > > > >> > > many request types need to change in order to incorporate the
> > new
> > > > >> member
> > > > >> > > name field.
> > > > >> > >
> > > > >> > > I also agree with what Jason and Guozhang have said regarding
> > > > >> timeouts.
> > > > >> > > Although semantically, it's easier to think of every operation
> > > > having
> > > > >> its
> > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > >> consolidation
> > > > >> > > seems preferable here. The definition of embedded protocols on
> > top
> > > > of
> > > > >> the
> > > > >> > > base group membership protocol for rebalancing gives enough
> > > > >> flexibility
> > > > >> > to
> > > > >> > > address such needs in each client component separately.
> > > > >> > >
> > > > >> > > Finally, some minor comments:
> > > > >> > > In a few places the new/proposed changes are referred to as
> > > > "current".
> > > > >> > > Which is a bit confusing considering that there is a protocol
> in
> > > > place
> > > > >> > > already, and by "current" someone might understand the
> existing
> > > one.
> > > > >> I'd
> > > > >> > > recommend using new/proposed or equivalent when referring to
> > > changes
> > > > >> > > introduced with KIP-345 and current/existing or equivalent
> when
> > > > >> referring
> > > > >> > > to existing behavior.
> > > > >> > >
> > > > >> > > There's the following sentence in the "Public Interfaces"
> > section:
> > > > >> > > "Since for many stateful consumer/stream applications, the
> state
> > > > >> > shuffling
> > > > >> > > is more painful than short time partial unavailability."
> > > > >> > > However, my understanding is that the changes proposed with
> > > KIP-345
> > > > >> will
> > > > >> > > not exploit any partial availability. A suggestion for dealing
> > > with
> > > > >> > > temporary imbalances has been made in "Incremental Cooperative
> > > > >> > Rebalancing"
> > > > >> > > which can work well with KIP-345, but here I don't see
> proposed
> > > > >> changes
> > > > >> > > that suggest that some resources (e.g. partitions) will keep
> > being
> > > > >> used
> > > > >> > > while others will not be utilized. Thus, you might want to
> > adjust
> > > > this
> > > > >> > > sentence. Correct me if I'm missing something related to that.
> > > > >> > >
> > > > >> > > In the rejected alternatives, under point 2) I read "we can
> copy
> > > the
> > > > >> > member
> > > > >> > > id to the config files". I believe it means to say "member
> name"
> > > > >> unless
> > > > >> > I'm
> > > > >> > > missing something about reusing member ids. Also below I read:
> > "By
> > > > >> > allowing
> > > > >> > > consumers to optionally specifying a member id" which probably
> > > > implies
> > > > >> > > "member name" again. In a sense this section highlights a
> > > potential
> > > > >> > > confusion between member name and member id. I wonder if we
> > could
> > > > >> come up
> > > > >> > > with a better term for the new field. StaticTag, StaticLabel,
> or
> > > > even
> > > > >> > > StaticName are some suggestions that could potentially help
> with
> > > > >> > confusion
> > > > >> > > between MemberId and MemberName and what corresponds to what.
> > But
> > > I
> > > > >> > > wouldn't like to disrupt the discussion with naming
> conventions
> > > too
> > > > >> much
> > > > >> > at
> > > > >> > > this point. I just mention it here as a thought.
> > > > >> > >
> > > > >> > > Looking forward to see the final details of this KIP. Great
> work
> > > so
> > > > >> far!
> > > > >> > >
> > > > >> > > Konstantine
> > > > >> > >
> > > > >> > >
> > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > bchen11@outlook.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Thanks Guozhang for the great summary here, and I have been
> > > > >> following
> > > > >> > up
> > > > >> > > > the action items here.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > timeout
> > > > and
> > > > >> > > > registration timeout. Great to see them being addressed in
> > > client
> > > > >> side!
> > > > >> > > >   2.  I double checked the design and I believe that it is
> ok
> > to
> > > > >> have
> > > > >> > > both
> > > > >> > > > static member and dynamic member co-exist in the same group.
> > So
> > > > the
> > > > >> > > upgrade
> > > > >> > > > shouldn't be destructive and we are removing the two
> > membership
> > > > >> > protocol
> > > > >> > > > switching APIs.
> > > > >> > > >   3.  I only have question about this one. I'm still reading
> > the
> > > > >> > > KafkaApis
> > > > >> > > > code here. Should I just use the same authorization logic
> for
> > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > > suggested,
> > > > >> > this
> > > > >> > > > feature could be better addressed in a separate KIP because
> it
> > > is
> > > > >> > pretty
> > > > >> > > > independent. I could start drafting the KIP once the current
> > > > >> proposal
> > > > >> > is
> > > > >> > > > approved.
> > > > >> > > >   5.  I believe that we don't need fencing in offset commit
> > > > request,
> > > > >> > > since
> > > > >> > > > duplicate member.name issue could be handled by join group
> > > > >> request. We
> > > > >> > > > shall reject join group with known member name but no member
> > id
> > > > >> (which
> > > > >> > > > means we already have an active member using this identity).
> > > > >> > > >   6.  I agree to remove that internal config once we move
> > > forward
> > > > >> with
> > > > >> > > > static membership. And I already removed the entire section
> > from
> > > > the
> > > > >> > KIP.
> > > > >> > > >
> > > > >> > > > Let me know if you have other concerns.
> > > > >> > > >
> > > > >> > > > Best,
> > > > >> > > > Boyang
> > > > >> > > > ________________________________
> > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > >> > > > To: dev
> > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > >> by
> > > > >> > > > specifying member id
> > > > >> > > >
> > > > >> > > > Hello Boyang,
> > > > >> > > >
> > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > appreciate
> > > > >> your
> > > > >> > > > patience answering to the feedbacks from the community. I'd
> > like
> > > > to
> > > > >> add
> > > > >> > > my
> > > > >> > > > 2cents here:
> > > > >> > > >
> > > > >> > > > 1. By introducing another two timeout configs,
> > > > registration_timeout
> > > > >> and
> > > > >> > > > expansion_timeout, we are effectively having four timeout
> > > configs:
> > > > >> > > session
> > > > >> > > > timeout, rebalance timeout (configured as "
> > max.poll.interval.ms
> > > "
> > > > on
> > > > >> > > client
> > > > >> > > > side), and these two. Interplaying these timeout configs can
> > be
> > > > >> quite
> > > > >> > > hard
> > > > >> > > > for users with such complexity, and hence I'm wondering if
> we
> > > can
> > > > >> > > simplify
> > > > >> > > > the situation with as less possible timeout configs as
> > possible.
> > > > >> Here
> > > > >> > is
> > > > >> > > a
> > > > >> > > > concrete suggestion I'd like propose:
> > > > >> > > >
> > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> addition
> > > to
> > > > >> the
> > > > >> > > > session_timeout for static members, we can just reuse the
> > > > >> > session_timeout
> > > > >> > > > and ask users to set it to a larger value when they are
> > > upgrading
> > > > a
> > > > >> > > dynamic
> > > > >> > > > client to a static client by setting the "member.name" at
> the
> > > > same
> > > > >> > time.
> > > > >> > > > By
> > > > >> > > > default, the broker-side min.session.timeout is 6 seconds
> and
> > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable to
> me
> > > (we
> > > > >> can
> > > > >> > of
> > > > >> > > > course modify this broker config to enlarge the valid
> interval
> > > if
> > > > we
> > > > >> > want
> > > > >> > > > in practice). And then we should also consider removing the
> > > > >> condition
> > > > >> > for
> > > > >> > > > marking a client as failed if the rebalance timeout has
> > reached
> > > > >> while
> > > > >> > the
> > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > session_timeout
> > > > >> > and
> > > > >> > > > rebalance_timeout are totally separated: the former is only
> > used
> > > > to
> > > > >> > > > determine if a consumer member of the group should be marked
> > as
> > > > >> failed
> > > > >> > > and
> > > > >> > > > kicked out of the group, and the latter is only used to
> > > determine
> > > > >> the
> > > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > > phase.
> > > > In
> > > > >> > > other
> > > > >> > > > words if a member did not send the JoinGroup in time of the
> > > > >> > > > rebalance_timeout, we still include it in the new generation
> > of
> > > > the
> > > > >> > group
> > > > >> > > > and use its old subscription info to send to leader for
> > > > assignment.
> > > > >> > Later
> > > > >> > > > if the member came back with HeartBeat request, we can still
> > > > follow
> > > > >> the
> > > > >> > > > normal path to bring it to the latest generation while
> > checking
> > > > that
> > > > >> > its
> > > > >> > > > sent JoinGroup request contains the same subscription info
> as
> > we
> > > > >> used
> > > > >> > to
> > > > >> > > > assign the partitions previously (which should be likely the
> > > case
> > > > in
> > > > >> > > > practice). In addition, we should let static members to not
> > send
> > > > the
> > > > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > > > static
> > > > >> > > member
> > > > >> > > > can only be leaving the group if its session has timed out,
> OR
> > > it
> > > > >> has
> > > > >> > > been
> > > > >> > > > indicated to not exist in the group any more (details
> below).
> > > > >> > > >
> > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > Cooperative
> > > > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> > > logic
> > > > >> at
> > > > >> > the
> > > > >> > > > application level, instead of at the protocol level. By
> doing
> > > this
> > > > >> we
> > > > >> > can
> > > > >> > > > also enable a few other optimizations, e.g. at the Streams
> > level
> > > > to
> > > > >> > first
> > > > >> > > > build up the state store as standby tasks and then trigger a
> > > > second
> > > > >> > > > rebalance to actually migrate the active tasks while keeping
> > the
> > > > >> actual
> > > > >> > > > rebalance latency and hence unavailability window to be
> small
> > (
> > > > >> > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=0rWXMbUYJSNUJ17u0%2BtvBDJADUuDpXYiJYyehb6fI%2Bc%3D&amp;reserved=0
> > > > >> > > ).
> > > > >> > > > I'd propose we align
> > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > >> > expansion_timeout
> > > > >> > > as
> > > > >> > > > part of the protocol layer, but only do that at the
> > > application's
> > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > > > still,
> > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > >> > > > <
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cd9af0652bf0d4d4a762d08d65560d756%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636790271763245858&amp;sdata=Kq4N4ICHQFjBIc%2BQ8rbBMnELb4FQcADzdB2URjvhqa8%3D&amp;reserved=0
> > > > >> > > >*"
> > > > >> > > > though as part of this KIP
> > > > >> > > > since we have discussed about its limit and think it is
> > actually
> > > > >> not a
> > > > >> > > very
> > > > >> > > > good design and could be replaced with client-side logic
> > above.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> this
> > > KIP.
> > > > >> More
> > > > >> > > > specifically, let's say after we have upgraded broker
> version
> > to
> > > > be
> > > > >> > able
> > > > >> > > to
> > > > >> > > > recognize the new versions of JoinGroup request and the
> admin
> > > > >> requests,
> > > > >> > > how
> > > > >> > > > should we upgrade the clients and enable static groups? On
> top
> > > of
> > > > my
> > > > >> > head
> > > > >> > > > if we do a rolling bounce in which we set the member.name
> > > config
> > > > as
> > > > >> > well
> > > > >> > > > as
> > > > >> > > > optionally increase the session.timeout config when we
> bounce
> > > each
> > > > >> > > > instance, then during this rolling bounces we will have a
> > group
> > > > >> > contained
> > > > >> > > > with both dynamic members and static members. It means that
> we
> > > > >> should
> > > > >> > > have
> > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > JoinGroup
> > > > >> > > requests
> > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > member.id"
> > > > >> > > mapping
> > > > >> > > > will only be partial at this scenario. Also could you
> describe
> > > if
> > > > >> the
> > > > >> > > > upgrade to the first version that support this feature would
> > > ever
> > > > >> get
> > > > >> > any
> > > > >> > > > benefits, or only the future upgrade path for rolling
> bounces
> > > > could
> > > > >> get
> > > > >> > > > benefits out of this feature?
> > > > >> > > >
> > > > >> > > > If that's the case and we will do 1) as suggested above, do
> we
> > > > still
> > > > >> > need
> > > > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > > > >> requests
> > > > >> > any
> > > > >> > > > more? Seems it is not necessary any more as we will only
> have
> > > the
> > > > >> > notion
> > > > >> > > of
> > > > >> > > > "dynamic or static members" that can co-exist in a group
> while
> > > > >> there no
> > > > >> > > > notion of "dynamic or static groups", and hence these two
> > > requests
> > > > >> are
> > > > >> > > not
> > > > >> > > > needed anymore.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 3. We need to briefly talk about the implications for ACL as
> > we
> > > > >> > introduce
> > > > >> > > > new admin requests that are related to a specific group.id.
> > For
> > > > >> > example,
> > > > >> > > > we
> > > > >> > > > need to make sure that whoever created the group or joined
> the
> > > > group
> > > > >> > can
> > > > >> > > > actually send admin requests for the group, otherwise the
> > > > >> application
> > > > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > > > cluster
> > > > >> > every
> > > > >> > > > time they want to send any admin requests for their groups
> > which
> > > > >> would
> > > > >> > be
> > > > >> > > > an operational nightmare.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 4. I like Jason's suggestion of adding an optional field for
> > the
> > > > >> list
> > > > >> > of
> > > > >> > > > member names, and I'm wondering if that can be done as part
> of
> > > the
> > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > members,
> > > > we
> > > > >> > will
> > > > >> > > > enforce a rebalance immediately since it indicates that some
> > > > static
> > > > >> > > member
> > > > >> > > > will be officially kicked out of the group and some new
> static
> > > > >> members
> > > > >> > > may
> > > > >> > > > be added. So back to 1.a) above, a static member can only be
> > > > kicked
> > > > >> out
> > > > >> > > of
> > > > >> > > > the group if a) its session (arguably long period of time)
> has
> > > > timed
> > > > >> > out,
> > > > >> > > > and b) this admin request explicitly state that it is no
> > longer
> > > > >> part of
> > > > >> > > the
> > > > >> > > > group. As for execution I'm fine with keeping it as a future
> > > work
> > > > of
> > > > >> > this
> > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > >> > > >
> > > > >> > > > Following are minor comments:
> > > > >> > > >
> > > > >> > > > 5. I'm not sure if we need to include "member.name" as part
> > of
> > > > the
> > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > memberId
> > > > >> plus
> > > > >> > > the
> > > > >> > > > generation number should be sufficient for fencing even with
> > > > static
> > > > >> > > > members.
> > > > >> > > >
> > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid
> of
> > > the
> > > > "
> > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > Guozhang
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> lindong28@gmail.com
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Hey Boyang,
> > > > >> > > > >
> > > > >> > > > > Thanks for the proposal! This is very useful. I have some
> > > > comments
> > > > >> > > below:
> > > > >> > > > >
> > > > >> > > > > 1) The motivation currently explicitly states that the
> goal
> > is
> > > > to
> > > > >> > > improve
> > > > >> > > > > performance for heavy state application. It seems that the
> > > > >> motivation
> > > > >> > > can
> > > > >> > > > > be stronger with the following use-case. Currently for
> > > > MirrorMaker
> > > > >> > > > cluster
> > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> > time
> > > to
> > > > >> > > rolling
> > > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > > process
> > > > >> > restart
> > > > >> > > > > will trigger a rebalance which currently pause the
> > consumption
> > > > of
> > > > >> the
> > > > >> > > all
> > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > stated
> > > in
> > > > >> this
> > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > specified
> > > > >> > > timeout
> > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > rebalance
> > > > >> > (e.g.
> > > > >> > > > for
> > > > >> > > > > leader restart) for the entire rolling bounce, which will
> > > > >> > significantly
> > > > >> > > > > improves the availability of the MirrorMaker pipeline. In
> my
> > > > >> opinion,
> > > > >> > > the
> > > > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance
> if
> > > the
> > > > >> > > consumer
> > > > >> > > > > process can be restarted within soon, which helps
> > performance
> > > > >> even if
> > > > >> > > > > overhead of state shuffling for a given process is small.
> > > > >> > > > >
> > > > >> > > > > 2) In order to simplify the KIP reading, can you follow
> the
> > > > >> writeup
> > > > >> > > style
> > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> > such
> > > as
> > > > >> new
> > > > >> > > > > configs (e.g. registration timeout), new request/response,
> > new
> > > > >> > > > AdminClient
> > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > Currently
> > > > >> some
> > > > >> > > of
> > > > >> > > > > these are specified in the Proposed Change section which
> > makes
> > > > it
> > > > >> a
> > > > >> > bit
> > > > >> > > > > inconvenient to understand the new interface that will be
> > > > exposed
> > > > >> to
> > > > >> > > > user.
> > > > >> > > > > Explanation of the current two-phase rebalance protocol
> > > probably
> > > > >> can
> > > > >> > be
> > > > >> > > > > moved out of public interface section.
> > > > >> > > > >
> > > > >> > > > > 3) There are currently two version of JoinGroupRequest in
> > the
> > > > KIP
> > > > >> and
> > > > >> > > > only
> > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > >> > > > >
> > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > > > >> rebalance
> > > > >> > > could
> > > > >> > > > > be helpful here, but we will make a call once we finished
> > the
> > > > >> major
> > > > >> > > > > implementation". So this seems to be still an open
> question
> > in
> > > > the
> > > > >> > > > current
> > > > >> > > > > design. We probably want to agree on this before voting
> for
> > > the
> > > > >> KIP.
> > > > >> > > > >
> > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > consumer.
> > > > Can
> > > > >> > you
> > > > >> > > > > specify the name of the config key and the default config
> > > value?
> > > > >> > > Possible
> > > > >> > > > > default values include empty string or null (similar to
> > > > >> > transaction.id
> > > > >> > > > in
> > > > >> > > > > producer config).
> > > > >> > > > >
> > > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > > persist
> > > > >> > member
> > > > >> > > > > name map, currently if consumer coordinator broker goes
> > > offline,
> > > > >> > > > rebalance
> > > > >> > > > > is triggered and consumers will try connect to the new
> > > > >> coordinator.
> > > > >> > If
> > > > >> > > > > these consumers can connect to the new coordinator within
> > > > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> > > that
> > > > >> > broker
> > > > >> > > > can
> > > > >> > > > > use a deterministic algorithm to determine the partition
> ->
> > > > >> > member_name
> > > > >> > > > > mapping, each consumer should get assigned the same set of
> > > > >> partitions
> > > > >> > > > > without requiring state shuffling. So it is not clear
> > whether
> > > we
> > > > >> > have a
> > > > >> > > > > strong use-case for this new logic. Can you help clarify
> > what
> > > is
> > > > >> the
> > > > >> > > > > benefit of using topic "static_member_map" to persist
> member
> > > > name
> > > > >> > map?
> > > > >> > > > >
> > > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > > config,
> > > > >> it is
> > > > >> > > > > mentioned that "we are using expansion timeout to replace
> > > > >> rebalance
> > > > >> > > > > timeout, which is configured by max.poll.intervals from
> > client
> > > > >> side,
> > > > >> > > and
> > > > >> > > > > using registration timeout to replace session timeout".
> > > > Currently
> > > > >> the
> > > > >> > > > > default max.poll.interval.ms is configured to be 5
> minutes
> > > and
> > > > >> there
> > > > >> > > > will
> > > > >> > > > > be only one rebalance if all new consumers can join
> within 5
> > > > >> minutes.
> > > > >> > > So
> > > > >> > > > it
> > > > >> > > > > is not clear whether we have a strong use-case for this
> new
> > > > >> config.
> > > > >> > Can
> > > > >> > > > you
> > > > >> > > > > explain what is the benefit of introducing this new
> config?
> > > > >> > > > >
> > > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > > version
> > > > >> of
> > > > >> > > > > protocol, we will also increase the join group request
> > version
> > > > to
> > > > >> v4
> > > > >> > > when
> > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> > > latest
> > > > >> (<
> > > > >> > > v4),
> > > > >> > > > > the join group request shall be downgraded to v3 without
> > > setting
> > > > >> the
> > > > >> > > > member
> > > > >> > > > > Id". It is probably simpler to just say that this feature
> is
> > > > >> enabled
> > > > >> > if
> > > > >> > > > > JoinGroupRequest V4 is supported on both client and broker
> > and
> > > > >> > > > MEMBER_NAME
> > > > >> > > > > is configured with non-empty string.
> > > > >> > > > >
> > > > >> > > > > 9) It is mentioned that broker may return
> > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > >> > > error
> > > > >> > > > > in OffsetCommitResponse for "commit requests under static
> > > > >> > membership".
> > > > >> > > > Can
> > > > >> > > > > you clarify how broker determines whether the commit
> request
> > > is
> > > > >> under
> > > > >> > > > > static membership?
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > > Dong
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > --
> > > > >> > > > -- Guozhang
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > --
> > > > >> > -Regards,
> > > > >> > Mayuresh R. Gharat
> > > > >> > (862) 250-7125
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -- Guozhang
>


--
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Guozhang Wang <wa...@gmail.com>.
Hi Boyang,

I was thinking that with the optional static members in the admin
ConsumerRebalanceRequest it should be sufficient to kick out the static
member before their session timeout (arguably long in practice) have not
reached. But now I see your concern is that in some situations the admin
operators may not even know the full list of static members, but ONLY know
which static member has failed and hence would like to kick out of the
group.

So I'd like to propose a slightly modified version of LeaveGroupRequest:
instead of letting the static member consumer client themselves to send the
request (which means we still need to have some hidden configs to turn it
off like we did today), how about just letting any other client to send
this request since the LeaveGroupRequest only requires group.id and
member.id? So back to your operational scenarios, if some static member has
been found crashed and it is not likely to comeback, or we simply want to
shrink the size of the group by shutting down some static members, we can
use an admin client to send the LeaveGroupRequest after the instance has
been completely shutdown or crashed to kick them out of the group and also
triggers the rebalance.

One issue though, is that users may not know the member id required in the
LeaveGroupRequest. To work around it we can add the `group.instance.id`
along with the member id as well and then allow member id null-able. The
coordinator logic would then be modified as 1) if member.id is specified,
ignore instance.id and always use member.id to find the member to kick out,
2) otherwise, try with the instance.id to find the corresponding member.id
and kick it out, 3) if none is found, reject with an error code.

So in sum the alternative changes are:

a) Modify LeaveGroupRequest to add group.instance.id
b) Modify coordinator logic to handle such request on the broker side.
c) Add a new API in AdminClient like "removeMemberFromGroup(groupId,
instanceId)" which will be translated as a LeaveGroupRequest.
d) [Optional] we can even batch the request by allowing
"removeMemberFromGroup(groupId, list[instanceId])" and then make `member.id`
and `instance.id` field of LeaveGroupRequest to be an array instead of a
single entry.
e) We can also remove the admin ConsumerRebalanceRequest as well for
simplicity (why not? paranoid of having as less request protocols as
possible :), as it is not needed anymore with the above proposal.


WDYT?


Guozhang

On Wed, Nov 28, 2018 at 5:34 AM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Guozhang and Mayuresh for the follow up! Answers are listed below.
>
>
> >  5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
>
> Great question. Let's use c1~c4 example here:
>
>   1.  Consumer c1, c2, c3, c4 in stable state
>   2.  c4 goes down and we detect this issue before session timeout through
> client monitoring. Initiate a ConsumerRebalanceRequest.
>   3.  A rebalance will be kicking off, and after rebalance timeout we
> shall keep the same assignment for c1~4, if the session timeout for c4
> hasn't reached
>   4.  Group back to stable with c1~4 (although c4 is actually offline)
>   5.  c4 session timeout finally reached: another rebalance triggered.
>
> For step 3, if session timeout triggered within rebalance timeout, only
> c1~3 will be participating in the rebalance. This is what we mean by saying
> "rebalance
> timeout shall not remove current members, only session timeout will do."
> As you could see this is not an ideal scenario: we trigger extra rebalance
> at step 5. In my reply to Guozhang I'm asking whether we should still use
> LeaveGroupRequest for static members to send a signal to broker saying "I'm
> currently offline", and when we send ConsumerRebalanceRequest to broker, we
> will actually kick off c4 because it says it's offline already, saving one
> or multiple additional rebalances later. This way the
> ConsumerRebalanceRequest will be more effective in making correct judgement
> on the group status since we have more feedback from client side.
>
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> Previous proposal should be suffice to answer this question 😊
>
> - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=lEv9SuBZgATGhOSe5zUr%2Fqudycoh%2FwfdM%2FhPH5Hp1N4%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> Thanks for catching it! And yes, we shall use one extra round-trip between
> consumer
> and broker to inform the new member id allocation.
>
> Next is the replies to Guozhang's comment:
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
> I'm also in favor of storing all the in-group static members. In fact we
> could reuse
> the static membership mapping to store this information. Do you think
> that we should let static member send leave group request to indicate
> their status of "leaving",
> and use ConsumerRebalanceRequest to trigger rebalance without them? I'm
> suggesting we should
> remove those members when kicking off rebalance since we are shutting them
> down already.
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
> Sure we have enough votes for ids 😊I will finalize the name to `
> group.instance.id`, does that
> sound good?
>
> Best,
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Wednesday, November 28, 2018 4:51 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Regarding Jason's question and Boyang's responses:
>
> 2) I once have a discussion about the LeaveGroupRequest for static members,
> and the reason for not having it for static members is that we'd need to
> make it a configurable behavior as well (i.e. the likelihood that a static
> member may shutdown but come back later may be even larger than the
> likelihood that a shutdown static member would not come back), and when a
> shutdown is complete the instance cannot tell whether or not it will come
> back by itself. And hence letting a third party (think: admin used by K8s
> plugins) issuing a request to indicate static member changes would be more
> plausible.
>
> I think having an optional list of all the static members that are still in
> the group, rather than the members to be removed since the latter looks a
> bit less flexible to me, in the request is a good idea (remember we allow a
> group to have both static and dynamic members at the same time, so when
> receiving the request, we will only do the diff and add / remove the static
> members directly only, while still let the dynamic members to try to
> re-join the group with the rebalance timeout).
>
> 3) personally I favor "ids" over "names" :) Since we already have some
> "ids" and hence it sounds more consistent, plus on the producer side we
> have a `transactional.id` whose semantics is a bit similar to this one,
> i.e. for unique distinguishment of a client which may comes and goes but
> need to be persist over multiple "instance life-times".
>
>
> Guozhang
>
>
> On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Thanks for the replies. Please find the follow up queries below.
> >
> >     5. Regarding "So in summary, *the member will only be removed due to
> > session timeout*. We shall remove it from both in-memory static member
> name
> > mapping and member list." If the rebalance is invoked manually using the
> > the admin apis, how long should the group coordinator wait for the
> members
> > of the group to send a JoinGroupRequest for participating in the
> rebalance?
> > How is a lagging consumer handled?
> > The plan is to disable member kick out when rebalance.timeout is reached,
> > so basically we are not "waiting" any
> > join group request from existing members; we shall just rebalance base on
> > what we currently have within the group
> > metadata. Lagging consumer will trigger rebalance later if session
> timeout
> > > rebalance timeout.
> >
> > >
> > Just wanted to understand this better. Lets take an example, say we have
> a
> > > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > > Everything is running fine and suddenly C4 host has issues and it goes
> > > down. Now we notice that we can still operate with c1, c2, c3 and don't
> > > want to wait for
> > > c4 to come back up. We use the admin api
> > > "invokeConsumerRebalance("GroupA")".
> > > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > > group again (in there heartBeatResponse) as first step of rebalance.
> > > Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> > > delayed. At this stage, if we are not "waiting" on any join group
> > request,
> > > few things can happen :
> > >
> > >    - c4's partitions are distributed only among c1,c2. c3 maintains its
> > >    original assignment. c1, c2 will start processing the newly assigned
> > >    partitions.
> > >
> > > OR
> > >
> > >    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
> > >    processing the newly assigned partitions. c3 gets to know about the
> > newly
> > >    assigned partitions later when it sends the JoinGroupRequest (which
> > was
> > >    delayed).
> > >
> > > OR
> > >
> > >    - Will the rebalance do a complete reassignment, where c1, c2, c3
> have
> > >    to give up there partitions and all the partitions belonging to c1,
> > c2, c3,
> > >    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
> > >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> > revoke
> > >    there partitions and rejoin the group.
> > >
> > > This is as per my understanding of how the KIP would work without
> > changing
> > > the underlying group coordination workflow. Please correct me if I
> > > misunderstood something here.
> > >
> >
> >
> > - When we say that we would use invokeConsumerRebalance(groupId) to down
> > scale, with the example in the above question, how will the
> > GroupCoordinator know that c4 should be kicked out of the group since we
> > are trying to invoke rebalance proactively without waiting for c4's
> session
> > time out to expire. Should there be a way of telling the GroupCoordinator
> > that consumer c4 has been kicked out of the groupId = "GroupA"?
> >
> > - Also it looks like the statement "If the `member.id` uses
> > UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> > the one within current map, if `group.member.name` is known. Also once
> we
> > are done with KIP-394
> > <
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=lEv9SuBZgATGhOSe5zUr%2Fqudycoh%2FwfdM%2FhPH5Hp1N4%3D&amp;reserved=0
> > >,
> > all the join group requests are requiring `member.id` to physically
> enter
> > the consumer group. This way the latest joined " is incomplete. Can you
> > take a look at this?
> > Also when we say "all the join group requests are requiring `member.id`
> to
> > physically enter the consumer group." because a newly started consumer
> will
> > not have a "member.id", I assume you mean, once the GroupCoordinator
> > assigns a member.id to the newly started consumer, it has to use it for
> > any
> > future JoinGroupRequests. Is my understanding correct?
> >
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer
> both
> > > in this reply.
> > >
> > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > >
> > > No, we shall only rely on broker to allocate member.id for the
> consumer
> > > instances. FYI, I already
> > >
> > > started the discussion thread for KIP-394 😊
> > >
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > >
> > > Yes, we could only use unknown member id. Actually this part of the
> > > proposal is outdated,
> > >
> > > let me do another audit of the whole doc. Basically, it is currently
> > > impossible to send `member.id`
> > >
> > > when consumer restarted. Sorry for the confusions!
> > >
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > >
> > > I would take Jason's advice that each time we have unknown member
> joining
> > > the group, the broker will
> > >
> > > always assign a new and unique id to track its identity. In this way,
> > > consumer with duplicate member name
> > >
> > > will be fenced.
> > >
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > >
> > > Like explained above, we shall not materialize the member.id. Instead
> we
> > > need to rely on broker to allocate
> > >
> > > a unique id for consumer just like what we have now.
> > >
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > >
> > > The plan is to disable member kick out when rebalance.timeout is
> reached,
> > > so basically we are not "waiting" any
> > >
> > > join group request from existing members; we shall just rebalance base
> on
> > > what we currently have within the group
> > >
> > > metadata. Lagging consumer will trigger rebalance later if session
> > timeout
> > > > rebalance timeout.
> > >
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > >
> > > Yep, it is. Add FindCoordinatorRequest logic to the script.
> > >
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > >
> > > Nah, I'm gonna remove it because it seems causing more confusion than
> > > making my assumption clear, which is
> > >
> > > "there could be other failure cases that I can't enumerate now" 😊
> > >
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > >
> > > Great suggestion! Added a simple rollback plan.
> > >
> > >
> > > Next is answering Jason's suggestions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > Yep, I like this idea! Keep giving out refresh member.id when facing
> > > anonymous request will definitely
> > >
> > > prevent processing bug due to duplicate consumers, however I don't
> think
> > I
> > > fully understand the 3rd paragraph where
> > >
> > > you mentioned  "There needs to be an expectation for static members
> that
> > > the member.id of a static member will
> > >
> > > not be changed except when a new member with the same member.name
> joins
> > > the group. "  How do you plan
> > > to know whether this member is new member or old member? I feel even
> with
> > > zombie consumer takes the ownership,
> > > it should be detected very quickly (as MISMATCH_ID exception trigger
> > > original consumer instance dies)
> > > and end user will start to fix it right away. Is there any similar
> logic
> > > we applied in fencing duplicate `transaction.id`?
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > This API is needed when we need to immediately trigger rebalance
> instead
> > > of waiting session timeout
> > >
> > > or rebalance timeout (Emergent scale up/down). It is very necessary to
> > > have it for
> > >
> > > management purpose because user could choose when to trigger rebalance
> > > pretty freely,
> > >
> > > gaining more client side control.
> > >
> > > In the meanwhile I see your point that we need to actually have the
> > > ability to kick out members that we plan
> > >
> > > to scale down fast (as rebalance timeout no longer kicks any offline
> > > member out of the group), I will think of adding an optional
> > >
> > > list of members that are ready to be removed.
> > >
> > > Another idea is to let static member send `LeaveGroupRequest` when they
> > > are going offline (either scale down or bouncing),
> > >
> > > and broker will cache this information as "OfflineMembers" without
> > > triggering rebalance. When handling ConsumerRebalanceRequest broker
> will
> > >
> > > kick the static members that are currently offline and trigger
> rebalance
> > > immediately. How does this plan sound?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> > > the namespace idea, and
> > >
> > > since we already use `member.name` in a lot of context, I decide to
> > > rename the config to `group.member.name`
> > >
> > > which should be sufficient for solving all the concerns we have now.
> > > Sounds good?
> > >
> > >
> > > Thank you for your great suggestions! Let me know if my reply makes
> sense
> > > her.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Jason Gustafson <ja...@confluent.io>
> > > Sent: Tuesday, November 27, 2018 7:51 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the updates. Looks like we're headed in the right direction
> > and
> > > clearly the interest that this KIP is receiving shows how strong the
> > > motivation is!
> > >
> > > I have a few questions:
> > >
> > > 1. This may be the same thing that Mayuresh is asking about. I think
> the
> > > suggestion in the KIP is that if a consumer sends JoinGroup with a
> member
> > > name, but no member id, then we will return the current member id
> > > associated with that name. It seems in this case that we wouldn't be
> able
> > > to protect from having two consumers active with the same configured
> > > member.name? For example, imagine that we had a consumer with
> > member.name
> > > =A
> > > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > > instance starts up with member.name=A. If it is also assigned
> member.id
> > =1,
> > > then how can we detect the zombie if it comes back to life? Both
> > instances
> > > will have the same member.id.
> > >
> > > The goal is to avoid a rebalance on a rolling restart, but we still
> need
> > to
> > > fence previous members. I am wondering if we can generate a new
> > member.id
> > > every time we receive a request from a static member with an unknown
> > member
> > > id. If the old instance with the same member.name attempts any
> > operation,
> > > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > > subscription of the new instance hasn't changed, then we can skip the
> > > rebalance and return the current assignment without forcing a
> rebalance.
> > >
> > > The trick to making this work is in the error handling of the zombie
> > > consumer. If the zombie simply resets its member.id and rejoins to
> get a
> > > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end
> up
> > > fencing the new member. We want to avoid this. There needs to be an
> > > expectation for static members that the member.id of a static member
> > will
> > > not be changed except when a new member with the same member.name
> joins
> > > the
> > > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for
> consumers
> > > with static member names.
> > >
> > > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As
> far
> > as
> > > I understand it, it is used for scaling down a consumer group and
> somehow
> > > bypasses normal session timeout expiration. I am wondering how critical
> > > this piece is and whether we can leave it for future work. If not, then
> > it
> > > would be helpful to elaborate on its implementation. How would the
> > > coordinator know which members to kick out of the group?
> > >
> > > 3. I've been holding back on mentioning this, but I think we should
> > > reconsider the name `member.name`. I think we want something that
> > suggests
> > > its expectation of uniqueness in the group. How about `
> group.instance.id
> > `
> > > to go along with `group.id`?
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > >
> > > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Thanks a lot for replying to all the queries and discussions here, so
> > > > patiently.
> > > > Really appreciate it.
> > > >
> > > > Had a few questions and suggestions after rereading the current
> version
> > > of
> > > > the KIP :
> > > >
> > > >
> > > >    1. Do you intend to have member.id is a static config like
> > > member.name
> > > >    after KIP-345 and KIP-394?
> > > >    2. Regarding "On client side, we add a new config called
> MEMBER_NAME
> > > in
> > > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME
> config
> > is
> > > > set,
> > > >    we will put it in the initial join group request to identify
> itself
> > > as a
> > > >    static member (static membership); otherwise, we will still send
> > > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > > (dynamic
> > > >    membership)."
> > > >       - What is the value of member_id sent in the first
> > JoinGroupRequest
> > > >       when member_name is set (using static rebalance)? Is it
> > > > UNKNOW_MEMBER_ID?
> > > >    3. Regarding "we are requiring member.id (if not unknown) to
> match
> > > the
> > > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The
> edge
> > > case
> > > >    that if we could have members with the same `member.name` (for
> > > example
> > > >    mis-configured instances with a valid member.id but added a used
> > > member
> > > >    name on runtime). When member name has duplicates, we could refuse
> > > join
> > > >    request from members with an outdated `member.id` (since we
> update
> > > the
> > > >    mapping upon each join group request). In an edge case where the
> > > client
> > > >    hits this exception in the response, it is suggesting that some
> > other
> > > >    consumer takes its spot."
> > > >       - The part of "some other consumer takes the spot" would be
> > > >       intentional, right? Also when you say " The edge case that if
> we
> > > >       could have members with the same `member.name` (for example
> > > >       mis-configured instances *with a valid member.id <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > > >
> > > > *but
> > > >       added a used member name on runtime).", what do you mean by
> > *valid
> > > >       member id* here? Does it mean that there exist a mapping of
> > > >       member.name to member.id like *MemberA -> id1* on the
> > > >       GroupCoordinator and this consumer is trying to join with *
> > > > member.name
> > > >       <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> > >
> > > = MemberB and member.id <
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > >
> > > =
> > > > id1 *
> > > >       ?
> > > >    4. Depending on your explanation for point 2 and the point 3 above
> > > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > > >    member_name but unknown member_id, if the consumer sends
> > > > "UNKNOW_MEMBER_ID"
> > > >    on the first JoinGroupRequest and relies on the GroupCoordinator
> to
> > > > give it
> > > >    a member_id, is the consumer suppose to remember member_id for
> > > >    joinGroupRequests? If yes, how are restarts handled?
> > > >    5. Regarding "So in summary, *the member will only be removed due
> to
> > > >    session timeout*. We shall remove it from both in-memory static
> > member
> > > >    name mapping and member list."
> > > >       - If the rebalance is invoked manually using the the admin
> apis,
> > > how
> > > >       long should the group coordinator wait for the members of the
> > > > group to send
> > > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > > lagging
> > > >       consumer handled?
> > > >    6. Another detail to take care is that we need to automatically
> take
> > > the
> > > >    hash of group id so that we know which broker to send this request
> > to.
> > > >       - I assume this should be same as the way we find the
> > coordinator,
> > > >       today right? If yes, should we specify it in the KIP ?
> > > >    7. Are there any specific failure scenarios when you say "other
> > > >    potential failure cases."? It would be good to mention them
> > > explicitly,
> > > > if
> > > >    you think there are any.
> > > >    8. It would be good to have a rollback plan as you have for roll
> > > forward
> > > >    in the KIP.
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > > gharatmayuresh15@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > > >
> > > > >> Hey Dong, thanks for the follow-up here!
> > > > >>
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >> Great question. I feel `member.name` is enough to explain itself,
> > it
> > > > >> seems not very
> > > > >> helpful to make the config name longer. Comparing `name` with `id`
> > > gives
> > > > >> user the
> > > > >> impression that they have the control over it with customized rule
> > > than
> > > > >> library decided.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >> We are not changing the default session timeout value. We are just
> > > > >> changing the
> > > > >> cap we are enforcing on the session timeout max value. So this
> > change
> > > is
> > > > >> not affecting
> > > > >> what kind of membership end user is using, and loosing the cap is
> > > giving
> > > > >> end user
> > > > >> more flexibility on trade-off between liveness and stability.
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error
> because
> > > with
> > > > >> the KIP-394<
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=45N6Tu9PTF3JL1BrS3PDDDMJKvsaO%2BL%2FE5Wdg1omDUI%3D&amp;reserved=0
> > > > >> >
> > > > >> we will automatically fence all join requests with
> > UNKNOWN_MEMBER_ID.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >> Same answer as question 3). This part of the logic shall be
> removed
> > > from
> > > > >> the proposal.
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >> Sounds good! Added both methods.
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=m7oZgEZRkfk3Ts2cVCImoeCjx3fjJ1Tt4mCF50loSjs%3D&amp;reserved=0
> > > > >> .
> > > > >> Added the script.
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >> Sounds good. Right now I feel for both static and dynamic
> membership
> > > it
> > > > is
> > > > >> more manageable to introduce the consumer rebalance method through
> > > admin
> > > > >> client API.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Sure, I will add more details to the API.
> > > > >>
> > > > >>
> > > > >> Thanks again for the helpful suggestions!
> > > > >>
> > > > >>
> > > > >> Best,
> > > > >> Boyang
> > > > >>
> > > > >> ________________________________
> > > > >> From: Dong Lin <li...@gmail.com>
> > > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > > >> To: dev
> > > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > > >> specifying member id
> > > > >>
> > > > >> Hey Boyang,
> > > > >>
> > > > >> Thanks for the update! Here are some followup comments:
> > > > >>
> > > > >> 1) It is not very clear to the user what is the difference between
> > > > >> member.name and client.id as both seems to be used to identify
> the
> > > > >> consumer. I am wondering if it would be more intuitive to name it
> > > > >> group.member.name (preferred choice since it matches the current
> > > > group.id
> > > > >> config name) or rebalance.member.name to explicitly show that the
> > id
> > > is
> > > > >> solely used for rebalance.
> > > > >>
> > > > >> 2) In the interface change section it is said that
> > > > >> GroupMaxSessionTimeoutMs
> > > > >> will be changed to 30 minutes. It seems to suggest that we will
> > change
> > > > the
> > > > >> default value of this config. It does not seem necessary to
> increase
> > > the
> > > > >> time of consumer failure detection when user doesn't use static
> > > > >> membership.
> > > > >> Also, say static membership is enabled, then this default config
> > > change
> > > > >> will cause a partition to be unavailable for consumption for 30
> > > minutes
> > > > if
> > > > >> there is hard consumer failure, which seems to be worse experience
> > > than
> > > > >> having unnecessary rebalance (when this timeout is small),
> > > particularly
> > > > >> for
> > > > >> new users of Kafka. Could you explain more why we should make this
> > > > change?
> > > > >>
> > > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > > DUPLICATE_STATIC_MEMBER
> > > > >> into one error? It seems that these two errors are currently
> handled
> > > by
> > > > >> the
> > > > >> consumer in the same way. And we don't also don't expect
> > > > >> MEMBER_ID_MISMATCH
> > > > >> to happen. Thus it is not clear what is the benefit of having two
> > > > errors.
> > > > >>
> > > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > > contains
> > > > >> member name which is already in the consumer group, however the
> > member
> > > > id
> > > > >> was missing". After a consumer is restarted, it will send a
> > > > >> JoinGroupRequest with an existing memberName (as the coordinator
> has
> > > not
> > > > >> expired this member from the memory) and memberId
> > > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> > persisted
> > > > >> across consumer restart in the consumer side). Does it mean that
> > > > >> JoinGroupRequest from a newly restarted consumer will always be
> > > rejected
> > > > >> until the sessionTimeoutMs has passed?
> > > > >>
> > > > >> 5) It seems that we always add two methods to the interface
> > > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> > and
> > > > the
> > > > >> other without option. Could this be specified in the interface
> > change
> > > > >> section?
> > > > >>
> > > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > > trigger
> > > > >> rebalance? If so, we probably want to specify the command line
> tool
> > > > >> interface similar to
> > > > >>
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=m7oZgEZRkfk3Ts2cVCImoeCjx3fjJ1Tt4mCF50loSjs%3D&amp;reserved=0
> > > > >> .
> > > > >>
> > > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > > meaning
> > > > >> of world "force" as compared to "trigger" or "invoke". And it
> seems
> > > > >> simpler
> > > > >> to allows this API to trigger rebalance regardless of whether
> > consumer
> > > > is
> > > > >> configured with memberName.
> > > > >>
> > > > >> 8) It is not very clear how the newly added AdminClient API
> trigger
> > > > >> rebalance. For example, does it send request? Can this be
> explained
> > in
> > > > the
> > > > >> KIP?
> > > > >>
> > > > >> Thanks,
> > > > >> Dong
> > > > >>
> > > > >>
> > > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > > > wrote:
> > > > >>
> > > > >> > Hey Mayuresh,
> > > > >> >
> > > > >> >
> > > > >> > thanks for your feedbacks! I will try do another checklist here.
> > > > >> >
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> >
> > > > >> > Yep it's true, we will prevent client from sending leave group
> > > request
> > > > >> > when they are set with `member.name`.
> > > > >> >
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > Good question, we will only kick off member due to session
> timeout
> > > > >> within
> > > > >> > static membership. Let me update the KIP to clearly assert that.
> > > > >> >
> > > > >> > > How is this case (missing member id) handled on the client
> side?
> > > > What
> > > > >> is
> > > > >> > the application that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > I have extended the two exceptions within join group response
> V4.
> > > > >> > Basically I define both corresponding actions to be immediate
> > > failing
> > > > >> > client application, because so far it is unknown what kind of
> > client
> > > > >> issue
> > > > >> > could trigger them. After the first version, we will keep
> enhance
> > > the
> > > > >> error
> > > > >> > handling logic!
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > The goal of extending session timeout cap is for users with good
> > > > client
> > > > >> > side monitoring tools that could auto-heal the dead consumers
> very
> > > > >> fast. So
> > > > >> > it is optional (and personal) to extend session timeout to a
> > > > reasonable
> > > > >> > number with different client scenarios.
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > Yep, there is a typo. Thanks for catching this!
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > No, there won't be. We want to limit the rebalance timeout
> > > > functionality
> > > > >> > to only use as a timer to
> > > > >> > end prepare rebalance stage. This way, late joining static
> members
> > > > will
> > > > >> > not trigger further rebalance
> > > > >> > as long as they are within session timeout. I added your
> highlight
> > > to
> > > > >> the
> > > > >> > KIP!
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > Updated the KIP: we shall not cover scale up case in 345,
> because
> > we
> > > > >> > believe client side could
> > > > >> > better handle this logic.
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > Great idea! In fact on top of static membership we could later
> > > > introduce
> > > > >> > APIs to set hard-coded
> > > > >> > client ids to the group and replace the dead host, or as you
> > > proposed
> > > > to
> > > > >> > define spare host as
> > > > >> > what I understood as hot backup. I will put both Jason and your
> > > > >> > suggestions into a separate section
> > > > >> > called "Future works". Note that this spare host idea may be
> also
> > > > >> solvable
> > > > >> > through rebalance protocol
> > > > >> > IMO.
> > > > >> >
> > > > >> > Thank you again for the great feedback!
> > > > >> >
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Boyang Chen <bc...@outlook.com>
> > > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> > email
> > > > on
> > > > >> my
> > > > >> > thread, so I will just do a checklist here!
> > > > >> >
> > > > >> >
> > > > >> > 1) The motivation currently explicitly states that the goal is
> to
> > > > >> improve
> > > > >> >
> > > > >> > performance for heavy state application. It seems that the
> > > motivation
> > > > >> can
> > > > >> >
> > > > >> > be stronger with the following use-case. Currently for
> MirrorMaker
> > > > >> cluster
> > > > >> >
> > > > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > > rolling
> > > > >> >
> > > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > > > restart
> > > > >> >
> > > > >> > will trigger a rebalance which currently pause the consumption
> of
> > > the
> > > > >> all
> > > > >> >
> > > > >> > partitions of the MirrorMaker cluster. With the change stated in
> > > this
> > > > >> >
> > > > >> > patch, as long as a MirrorMaker can restart within the specified
> > > > timeout
> > > > >> >
> > > > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> > > (e.g.
> > > > >> for
> > > > >> >
> > > > >> > leader restart) for the entire rolling bounce, which will
> > > > significantly
> > > > >> >
> > > > >> > improves the availability of the MirrorMaker pipeline. In my
> > > opinion,
> > > > >> the
> > > > >> >
> > > > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > > >> consumer
> > > > >> >
> > > > >> > process can be restarted within soon, which helps performance
> even
> > > if
> > > > >> >
> > > > >> > overhead of state shuffling for a given process is small.
> > > > >> >
> > > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > > making
> > > > >> the
> > > > >> > motivation more solid!
> > > > >> >
> > > > >> > 2) In order to simplify the KIP reading, can you follow the
> > writeup
> > > > >> style
> > > > >> > of other KIP (e.g. KIP-98) and list the interface change such as
> > new
> > > > >> > configs (e.g. registration timeout), new request/response, new
> > > > >> AdminClient
> > > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> > > some
> > > > of
> > > > >> > these are specified in the Proposed Change section which makes
> it
> > a
> > > > bit
> > > > >> > inconvenient to understand the new interface that will be
> exposed
> > to
> > > > >> user.
> > > > >> > Explanation of the current two-phase rebalance protocol probably
> > can
> > > > be
> > > > >> > moved out of public interface section.
> > > > >> > This is a great suggestion! I just consolidated all the public
> API
> > > > >> > changes, and the whole KIP
> > > > >> > looks much more organized!
> > > > >> >
> > > > >> > 3) There are currently two version of JoinGroupRequest in the
> KIP
> > > and
> > > > >> only
> > > > >> > one of them has field memberId. This seems confusing.
> > > > >> > Yep, I already found this issue and fixed it.
> > > > >> >
> > > > >> > 4) It is mentioned in the KIP that "An admin API to force
> > rebalance
> > > > >> could
> > > > >> > be helpful here, but we will make a call once we finished the
> > major
> > > > >> > implementation". So this seems to be still an open question in
> the
> > > > >> current
> > > > >> > design. We probably want to agree on this before voting for the
> > KIP.
> > > > >> > We have finalized the idea that this API is needed.
> > > > >> >
> > > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> Can
> > > you
> > > > >> > specify the name of the config key and the default config value?
> > > > >> Possible
> > > > >> > default values include empty string or null (similar to
> > > > transaction.id<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ecs5ZNM3ZUKB7W3u6yzmihcZrTw%2BOHnF16oRPLOlYB8%3D&amp;reserved=0
> > > > >> >
> > > > >> > in
> > > > >> > producer config).
> > > > >> > I have defined the `member.name` in "New configuration"
> section.
> > > > >> >
> > > > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > > > member
> > > > >> > name map, currently if consumer coordinator broker goes offline,
> > > > >> rebalance
> > > > >> > is triggered and consumers will try connect to the new
> > coordinator.
> > > If
> > > > >> > these consumers can connect to the new coordinator within
> > > > >> > max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ufEq5FifjLJYW0N0bKABNETkGnB9PgcYitoVJrYrr%2Bw%3D&amp;reserved=0
> > > > >> >
> > > > >> > which by default is 5 minutes, given that broker can
> > > > >> > use a deterministic algorithm to determine the partition ->
> > > > member_name
> > > > >> > mapping, each consumer should get assigned the same set of
> > > partitions
> > > > >> > without requiring state shuffling. So it is not clear whether we
> > > have
> > > > a
> > > > >> > strong use-case for this new logic. Can you help clarify what is
> > the
> > > > >> > benefit of using topic "static_member_map" to persist member
> name
> > > map?
> > > > >> > I have discussed with Guozhang offline, and I believe reusing
> the
> > > > >> current
> > > > >> > `_consumer_offsets`
> > > > >> > topic is a better and unified solution.
> > > > >> >
> > > > >> > 7) Regarding the introduction of the expensionTimeoutMs config,
> it
> > > is
> > > > >> > mentioned that "we are using expansion timeout to replace
> > rebalance
> > > > >> > timeout, which is configured by max.poll.intervals from client
> > side,
> > > > and
> > > > >> > using registration timeout to replace session timeout".
> Currently
> > > the
> > > > >> > default max.poll.interval.ms<
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ufEq5FifjLJYW0N0bKABNETkGnB9PgcYitoVJrYrr%2Bw%3D&amp;reserved=0
> > > > >> >
> > > > >> > is configured to be 5 minutes and there will
> > > > >> > be only one rebalance if all new consumers can join within 5
> > > minutes.
> > > > >> So it
> > > > >> > is not clear whether we have a strong use-case for this new
> > config.
> > > > Can
> > > > >> you
> > > > >> > explain what is the benefit of introducing this new config?
> > > > >> > Previously our goal is to use expansion timeout as a workaround
> > for
> > > > >> > triggering multiple
> > > > >> > rebalances when scaling up members are not joining at the same
> > time.
> > > > It
> > > > >> is
> > > > >> > decided to
> > > > >> > be addressed by client side protocol change, so we will not
> > > introduce
> > > > >> > expansion timeout.
> > > > >> >
> > > > >> > 8) It is mentioned that "To distinguish between previous version
> > of
> > > > >> > protocol, we will also increase the join group request version
> to
> > v4
> > > > >> when
> > > > >> > MEMBER_NAME is set" and "If the broker version is not the latest
> > (<
> > > > v4),
> > > > >> > the join group request shall be downgraded to v3 without setting
> > the
> > > > >> member
> > > > >> > Id". It is probably simpler to just say that this feature is
> > enabled
> > > > if
> > > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > > >> MEMBER_NAME
> > > > >> > is configured with non-empty string.
> > > > >> > Yep, addressed this!
> > > > >> >
> > > > >> > 9) It is mentioned that broker may return
> > NO_STATIC_MEMBER_INFO_SET
> > > > >> error
> > > > >> > in OffsetCommitResponse for "commit requests under static
> > > membership".
> > > > >> Can
> > > > >> > you clarify how broker determines whether the commit request is
> > > under
> > > > >> > static membership?
> > > > >> >
> > > > >> > We have agreed that commit request shouldn't be affected by the
> > new
> > > > >> > membership, thus
> > > > >> > removing it here. Thanks for catching this!
> > > > >> >
> > > > >> > Let me know if you have further suggestions or concerns. Thank
> you
> > > for
> > > > >> > your valuable feedback
> > > > >> > to help me design the KIP better! (And I will try to address
> your
> > > > >> > feedbacks in next round Mayuresh ??)
> > > > >> >
> > > > >> > Best,
> > > > >> > Boyang
> > > > >> > ________________________________
> > > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > > >> > To: dev@kafka.apache.org
> > > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > by
> > > > >> > specifying member id
> > > > >> >
> > > > >> > Hi Boyang,
> > > > >> >
> > > > >> > Thanks for updating the KIP. This is a step good direction for
> > > > stateful
> > > > >> > applications and also mirroring applications whose latency is
> > > affected
> > > > >> due
> > > > >> > to the rebalance issues that we have today.
> > > > >> >
> > > > >> > I had a few questions on the current version of the KIP :
> > > > >> > For the effectiveness of the KIP, consumer with member.name set
> > > will
> > > > >> *not
> > > > >> > send leave group request* when they go offline
> > > > >> >
> > > > >> > > By this you mean, even if the application has not called
> > > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > > sending
> > > > >> the
> > > > >> > > LeaveGroup request, right?
> > > > >> > >
> > > > >> >
> > > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > > member.id
> > > > }
> > > > >> to
> > > > >> > track member uniqueness.
> > > > >> >
> > > > >> > > When is the member.name removed from this map?
> > > > >> > >
> > > > >> >
> > > > >> > Member.id must be set if the *member.name <
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> > > > >> >
> > > > >> > *is already
> > > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > > >> >
> > > > >> > > How is this case handled on the client side? What is the
> > > application
> > > > >> that
> > > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > > >> > >
> > > > >> >
> > > > >> > Session timeout is the timeout we will trigger rebalance when a
> > > member
> > > > >> goes
> > > > >> > offline for too long (not sending heartbeat request). To make
> > static
> > > > >> > membership effective, we should increase the default max session
> > > > >> timeout to
> > > > >> > 30 min so that end user could config it freely.
> > > > >> >
> > > > >> > > This would mean that it might take more time to detect unowned
> > > topic
> > > > >> > > partitions and may cause delay for applications that perform
> > data
> > > > >> > mirroring
> > > > >> > > tasks. I discussed this with our sre and we have a suggestion
> to
> > > > make
> > > > >> > here
> > > > >> > > as listed below separately.
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups
> > > > >> >
> > > > >> > > you meant remove unjoined members of the group, right ?
> > > > >> > >
> > > > >> >
> > > > >> > Currently there is a config called *rebalance timeout* which is
> > > > >> configured
> > > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > > interval
> > > > >> is
> > > > >> > because consumer could only send request within the call of
> poll()
> > > and
> > > > >> we
> > > > >> > want to wait sufficient time for the join group request. When
> > > reaching
> > > > >> > rebalance timeout, the group will move towards
> completingRebalance
> > > > stage
> > > > >> > and remove unjoined groups. This is actually conflicting with
> the
> > > > >> design of
> > > > >> > static membership, because those temporarily unavailable members
> > > will
> > > > >> > potentially reattempt the join group and trigger extra
> rebalances.
> > > > >> > Internally we would optimize this logic by having rebalance
> > timeout
> > > > >> only in
> > > > >> > charge of stopping prepare rebalance stage, without removing
> > > > >> non-responsive
> > > > >> > members immediately.
> > > > >> >
> > > > >> > > What do you mean by " Internally we would optimize this logic
> by
> > > > >> having
> > > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > > stage,
> > > > >> > > without removing non-responsive members immediately." There
> > would
> > > > not
> > > > >> be
> > > > >> > a
> > > > >> > > full rebalance if the lagging consumer sent a JoinGroup
> request
> > > > later,
> > > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > > >> > >
> > > > >> >
> > > > >> > Scale Up
> > > > >> >
> > > > >> > > The KIP talks about scale up scenario but its not quite clear
> > how
> > > we
> > > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > > adding
> > > > >> > status
> > > > >> > > "learner" ?. Can you shed more light on how this is handled in
> > the
> > > > >> KIP,
> > > > >> > if
> > > > >> > > its handled?
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > *Discussion*
> > > > >> > Larger session timeouts causing latency rise for getting data
> for
> > > > >> un-owned
> > > > >> > topic partitions :
> > > > >> >
> > > > >> > > I think Jason had brought this up earlier about having a way
> to
> > > say
> > > > >> how
> > > > >> > > many members/consumer hosts are you choosing to be in the
> > consumer
> > > > >> group.
> > > > >> > > If we can do this, then in case of mirroring applications we
> can
> > > do
> > > > >> this
> > > > >> > :
> > > > >> > > Lets say we have a mirroring application that consumes from
> > Kafka
> > > > >> cluster
> > > > >> > > A and produces to Kafka cluster B.
> > > > >> > > Depending on the data and the Kafka cluster configuration,
> Kafka
> > > > >> service
> > > > >> > > providers can set a mirroring group saying that it will take,
> > for
> > > > >> example
> > > > >> > > 300 consumer hosts/members to achieve the desired throughput
> and
> > > > >> latency
> > > > >> > > for mirroring and can have additional 10 consumer hosts as
> spare
> > > in
> > > > >> the
> > > > >> > > same group.
> > > > >> > > So when the first 300 members/consumers to join the group will
> > > start
> > > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > > >> > > The remaining 10 consumer members can sit idle.
> > > > >> > > The moment one of the consumer (for example: consumer number
> 54)
> > > > from
> > > > >> the
> > > > >> > > first 300 members go out of the group (crossed session
> timeout),
> > > it
> > > > >> (the
> > > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > > >> consumer
> > > > >> > > member 54 to one of the spare hosts.
> > > > >> > > Once the consumer member 54 comes back up, it can start as
> > being a
> > > > >> part
> > > > >> > of
> > > > >> > > the spare pool.
> > > > >> > > This enables us to have lower session timeouts and low latency
> > > > >> mirroring,
> > > > >> > > in cases where the service providers are OK with having spare
> > > hosts.
> > > > >> > > This would mean that we would tolerate n consumer members
> > leaving
> > > > and
> > > > >> > > rejoining the group and still provide low latency as long as n
> > <=
> > > > >> number
> > > > >> > of
> > > > >> > > spare consumers.
> > > > >> > > If there are no spare host available, we can get back to the
> > idea
> > > as
> > > > >> > > described in the KIP.
> > > > >> > >
> > > > >> >
> > > > >> > Thanks,
> > > > >> >
> > > > >> > Mayuresh
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > > >> > konstantine@confluent.io> wrote:
> > > > >> >
> > > > >> > > Hi Boyang.
> > > > >> > >
> > > > >> > > Thanks for preparing this KIP! It is making good progress and
> > will
> > > > be
> > > > >> a
> > > > >> > > great improvement for stateful Kafka applications.
> > > > >> > >
> > > > >> > > Apologies for my late reply, I was away for a while. Lots of
> > great
> > > > >> > comments
> > > > >> > > so far, so I'll probably second most of them in what I suggest
> > > below
> > > > >> at
> > > > >> > > this point.
> > > > >> > >
> > > > >> > > When I first read the KIP, I wanted to start at the end with
> > > > something
> > > > >> > that
> > > > >> > > wasn't highlighted a lot. That was the topic related to
> handling
> > > > >> > duplicate
> > > > >> > > members. I see now that the initial suggestion of handling
> this
> > > > >> situation
> > > > >> > > during offset commit has been removed, and I agree with that.
> > > Issues
> > > > >> > > related to membership seem to be handled better when the
> member
> > > > joins
> > > > >> the
> > > > >> > > group rather than when it tries to commit offsets. This also
> > > > >> simplifies
> > > > >> > how
> > > > >> > > many request types need to change in order to incorporate the
> > new
> > > > >> member
> > > > >> > > name field.
> > > > >> > >
> > > > >> > > I also agree with what Jason and Guozhang have said regarding
> > > > >> timeouts.
> > > > >> > > Although semantically, it's easier to think of every operation
> > > > having
> > > > >> its
> > > > >> > > own timeout, operationally this can become a burden. Thus,
> > > > >> consolidation
> > > > >> > > seems preferable here. The definition of embedded protocols on
> > top
> > > > of
> > > > >> the
> > > > >> > > base group membership protocol for rebalancing gives enough
> > > > >> flexibility
> > > > >> > to
> > > > >> > > address such needs in each client component separately.
> > > > >> > >
> > > > >> > > Finally, some minor comments:
> > > > >> > > In a few places the new/proposed changes are referred to as
> > > > "current".
> > > > >> > > Which is a bit confusing considering that there is a protocol
> in
> > > > place
> > > > >> > > already, and by "current" someone might understand the
> existing
> > > one.
> > > > >> I'd
> > > > >> > > recommend using new/proposed or equivalent when referring to
> > > changes
> > > > >> > > introduced with KIP-345 and current/existing or equivalent
> when
> > > > >> referring
> > > > >> > > to existing behavior.
> > > > >> > >
> > > > >> > > There's the following sentence in the "Public Interfaces"
> > section:
> > > > >> > > "Since for many stateful consumer/stream applications, the
> state
> > > > >> > shuffling
> > > > >> > > is more painful than short time partial unavailability."
> > > > >> > > However, my understanding is that the changes proposed with
> > > KIP-345
> > > > >> will
> > > > >> > > not exploit any partial availability. A suggestion for dealing
> > > with
> > > > >> > > temporary imbalances has been made in "Incremental Cooperative
> > > > >> > Rebalancing"
> > > > >> > > which can work well with KIP-345, but here I don't see
> proposed
> > > > >> changes
> > > > >> > > that suggest that some resources (e.g. partitions) will keep
> > being
> > > > >> used
> > > > >> > > while others will not be utilized. Thus, you might want to
> > adjust
> > > > this
> > > > >> > > sentence. Correct me if I'm missing something related to that.
> > > > >> > >
> > > > >> > > In the rejected alternatives, under point 2) I read "we can
> copy
> > > the
> > > > >> > member
> > > > >> > > id to the config files". I believe it means to say "member
> name"
> > > > >> unless
> > > > >> > I'm
> > > > >> > > missing something about reusing member ids. Also below I read:
> > "By
> > > > >> > allowing
> > > > >> > > consumers to optionally specifying a member id" which probably
> > > > implies
> > > > >> > > "member name" again. In a sense this section highlights a
> > > potential
> > > > >> > > confusion between member name and member id. I wonder if we
> > could
> > > > >> come up
> > > > >> > > with a better term for the new field. StaticTag, StaticLabel,
> or
> > > > even
> > > > >> > > StaticName are some suggestions that could potentially help
> with
> > > > >> > confusion
> > > > >> > > between MemberId and MemberName and what corresponds to what.
> > But
> > > I
> > > > >> > > wouldn't like to disrupt the discussion with naming
> conventions
> > > too
> > > > >> much
> > > > >> > at
> > > > >> > > this point. I just mention it here as a thought.
> > > > >> > >
> > > > >> > > Looking forward to see the final details of this KIP. Great
> work
> > > so
> > > > >> far!
> > > > >> > >
> > > > >> > > Konstantine
> > > > >> > >
> > > > >> > >
> > > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> > bchen11@outlook.com>
> > > > >> wrote:
> > > > >> > >
> > > > >> > > > Thanks Guozhang for the great summary here, and I have been
> > > > >> following
> > > > >> > up
> > > > >> > > > the action items here.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >   1.  I already updated the KIP to remove the expansion
> > timeout
> > > > and
> > > > >> > > > registration timeout. Great to see them being addressed in
> > > client
> > > > >> side!
> > > > >> > > >   2.  I double checked the design and I believe that it is
> ok
> > to
> > > > >> have
> > > > >> > > both
> > > > >> > > > static member and dynamic member co-exist in the same group.
> > So
> > > > the
> > > > >> > > upgrade
> > > > >> > > > shouldn't be destructive and we are removing the two
> > membership
> > > > >> > protocol
> > > > >> > > > switching APIs.
> > > > >> > > >   3.  I only have question about this one. I'm still reading
> > the
> > > > >> > > KafkaApis
> > > > >> > > > code here. Should I just use the same authorization logic
> for
> > > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > > suggested,
> > > > >> > this
> > > > >> > > > feature could be better addressed in a separate KIP because
> it
> > > is
> > > > >> > pretty
> > > > >> > > > independent. I could start drafting the KIP once the current
> > > > >> proposal
> > > > >> > is
> > > > >> > > > approved.
> > > > >> > > >   5.  I believe that we don't need fencing in offset commit
> > > > request,
> > > > >> > > since
> > > > >> > > > duplicate member.name issue could be handled by join group
> > > > >> request. We
> > > > >> > > > shall reject join group with known member name but no member
> > id
> > > > >> (which
> > > > >> > > > means we already have an active member using this identity).
> > > > >> > > >   6.  I agree to remove that internal config once we move
> > > forward
> > > > >> with
> > > > >> > > > static membership. And I already removed the entire section
> > from
> > > > the
> > > > >> > KIP.
> > > > >> > > >
> > > > >> > > > Let me know if you have other concerns.
> > > > >> > > >
> > > > >> > > > Best,
> > > > >> > > > Boyang
> > > > >> > > > ________________________________
> > > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > >> > > > To: dev
> > > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > >> by
> > > > >> > > > specifying member id
> > > > >> > > >
> > > > >> > > > Hello Boyang,
> > > > >> > > >
> > > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > > appreciate
> > > > >> your
> > > > >> > > > patience answering to the feedbacks from the community. I'd
> > like
> > > > to
> > > > >> add
> > > > >> > > my
> > > > >> > > > 2cents here:
> > > > >> > > >
> > > > >> > > > 1. By introducing another two timeout configs,
> > > > registration_timeout
> > > > >> and
> > > > >> > > > expansion_timeout, we are effectively having four timeout
> > > configs:
> > > > >> > > session
> > > > >> > > > timeout, rebalance timeout (configured as "
> > max.poll.interval.ms
> > > "
> > > > on
> > > > >> > > client
> > > > >> > > > side), and these two. Interplaying these timeout configs can
> > be
> > > > >> quite
> > > > >> > > hard
> > > > >> > > > for users with such complexity, and hence I'm wondering if
> we
> > > can
> > > > >> > > simplify
> > > > >> > > > the situation with as less possible timeout configs as
> > possible.
> > > > >> Here
> > > > >> > is
> > > > >> > > a
> > > > >> > > > concrete suggestion I'd like propose:
> > > > >> > > >
> > > > >> > > > 1.a) Instead of introducing a registration_timeout in
> addition
> > > to
> > > > >> the
> > > > >> > > > session_timeout for static members, we can just reuse the
> > > > >> > session_timeout
> > > > >> > > > and ask users to set it to a larger value when they are
> > > upgrading
> > > > a
> > > > >> > > dynamic
> > > > >> > > > client to a static client by setting the "member.name" at
> the
> > > > same
> > > > >> > time.
> > > > >> > > > By
> > > > >> > > > default, the broker-side min.session.timeout is 6 seconds
> and
> > > > >> > > > max.session.timeout is 5 minutes, which seems reasonable to
> me
> > > (we
> > > > >> can
> > > > >> > of
> > > > >> > > > course modify this broker config to enlarge the valid
> interval
> > > if
> > > > we
> > > > >> > want
> > > > >> > > > in practice). And then we should also consider removing the
> > > > >> condition
> > > > >> > for
> > > > >> > > > marking a client as failed if the rebalance timeout has
> > reached
> > > > >> while
> > > > >> > the
> > > > >> > > > JoinGroup was not received, so that the semantics of
> > > > session_timeout
> > > > >> > and
> > > > >> > > > rebalance_timeout are totally separated: the former is only
> > used
> > > > to
> > > > >> > > > determine if a consumer member of the group should be marked
> > as
> > > > >> failed
> > > > >> > > and
> > > > >> > > > kicked out of the group, and the latter is only used to
> > > determine
> > > > >> the
> > > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > > phase.
> > > > In
> > > > >> > > other
> > > > >> > > > words if a member did not send the JoinGroup in time of the
> > > > >> > > > rebalance_timeout, we still include it in the new generation
> > of
> > > > the
> > > > >> > group
> > > > >> > > > and use its old subscription info to send to leader for
> > > > assignment.
> > > > >> > Later
> > > > >> > > > if the member came back with HeartBeat request, we can still
> > > > follow
> > > > >> the
> > > > >> > > > normal path to bring it to the latest generation while
> > checking
> > > > that
> > > > >> > its
> > > > >> > > > sent JoinGroup request contains the same subscription info
> as
> > we
> > > > >> used
> > > > >> > to
> > > > >> > > > assign the partitions previously (which should be likely the
> > > case
> > > > in
> > > > >> > > > practice). In addition, we should let static members to not
> > send
> > > > the
> > > > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > > > static
> > > > >> > > member
> > > > >> > > > can only be leaving the group if its session has timed out,
> OR
> > > it
> > > > >> has
> > > > >> > > been
> > > > >> > > > indicated to not exist in the group any more (details
> below).
> > > > >> > > >
> > > > >> > > > 1.b) We have a parallel discussion about Incremental
> > Cooperative
> > > > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> > > logic
> > > > >> at
> > > > >> > the
> > > > >> > > > application level, instead of at the protocol level. By
> doing
> > > this
> > > > >> we
> > > > >> > can
> > > > >> > > > also enable a few other optimizations, e.g. at the Streams
> > level
> > > > to
> > > > >> > first
> > > > >> > > > build up the state store as standby tasks and then trigger a
> > > > second
> > > > >> > > > rebalance to actually migrate the active tasks while keeping
> > the
> > > > >> actual
> > > > >> > > > rebalance latency and hence unavailability window to be
> small
> > (
> > > > >> > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=eHHZXH6id%2FihibQ9640lWEsjz3VKROW9JfvTni4HO2s%3D&amp;reserved=0
> > > > >> > > ).
> > > > >> > > > I'd propose we align
> > > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > > >> > expansion_timeout
> > > > >> > > as
> > > > >> > > > part of the protocol layer, but only do that at the
> > > application's
> > > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > > > still,
> > > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > >> > > > <
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=8iZF4R9mdfnEH4RPPa%2ButUWhrBVRYuRVHi98RsbcRPQ%3D&amp;reserved=0
> > > > >> > > >*"
> > > > >> > > > though as part of this KIP
> > > > >> > > > since we have discussed about its limit and think it is
> > actually
> > > > >> not a
> > > > >> > > very
> > > > >> > > > good design and could be replaced with client-side logic
> > above.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 2. I'd like to see your thoughts on the upgrade path for
> this
> > > KIP.
> > > > >> More
> > > > >> > > > specifically, let's say after we have upgraded broker
> version
> > to
> > > > be
> > > > >> > able
> > > > >> > > to
> > > > >> > > > recognize the new versions of JoinGroup request and the
> admin
> > > > >> requests,
> > > > >> > > how
> > > > >> > > > should we upgrade the clients and enable static groups? On
> top
> > > of
> > > > my
> > > > >> > head
> > > > >> > > > if we do a rolling bounce in which we set the member.name
> > > config
> > > > as
> > > > >> > well
> > > > >> > > > as
> > > > >> > > > optionally increase the session.timeout config when we
> bounce
> > > each
> > > > >> > > > instance, then during this rolling bounces we will have a
> > group
> > > > >> > contained
> > > > >> > > > with both dynamic members and static members. It means that
> we
> > > > >> should
> > > > >> > > have
> > > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > > JoinGroup
> > > > >> > > requests
> > > > >> > > > from dynamic members), and hence the "member.name" -> "
> > > member.id"
> > > > >> > > mapping
> > > > >> > > > will only be partial at this scenario. Also could you
> describe
> > > if
> > > > >> the
> > > > >> > > > upgrade to the first version that support this feature would
> > > ever
> > > > >> get
> > > > >> > any
> > > > >> > > > benefits, or only the future upgrade path for rolling
> bounces
> > > > could
> > > > >> get
> > > > >> > > > benefits out of this feature?
> > > > >> > > >
> > > > >> > > > If that's the case and we will do 1) as suggested above, do
> we
> > > > still
> > > > >> > need
> > > > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > > > >> requests
> > > > >> > any
> > > > >> > > > more? Seems it is not necessary any more as we will only
> have
> > > the
> > > > >> > notion
> > > > >> > > of
> > > > >> > > > "dynamic or static members" that can co-exist in a group
> while
> > > > >> there no
> > > > >> > > > notion of "dynamic or static groups", and hence these two
> > > requests
> > > > >> are
> > > > >> > > not
> > > > >> > > > needed anymore.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 3. We need to briefly talk about the implications for ACL as
> > we
> > > > >> > introduce
> > > > >> > > > new admin requests that are related to a specific group.id.
> > For
> > > > >> > example,
> > > > >> > > > we
> > > > >> > > > need to make sure that whoever created the group or joined
> the
> > > > group
> > > > >> > can
> > > > >> > > > actually send admin requests for the group, otherwise the
> > > > >> application
> > > > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > > > cluster
> > > > >> > every
> > > > >> > > > time they want to send any admin requests for their groups
> > which
> > > > >> would
> > > > >> > be
> > > > >> > > > an operational nightmare.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > 4. I like Jason's suggestion of adding an optional field for
> > the
> > > > >> list
> > > > >> > of
> > > > >> > > > member names, and I'm wondering if that can be done as part
> of
> > > the
> > > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> > members,
> > > > we
> > > > >> > will
> > > > >> > > > enforce a rebalance immediately since it indicates that some
> > > > static
> > > > >> > > member
> > > > >> > > > will be officially kicked out of the group and some new
> static
> > > > >> members
> > > > >> > > may
> > > > >> > > > be added. So back to 1.a) above, a static member can only be
> > > > kicked
> > > > >> out
> > > > >> > > of
> > > > >> > > > the group if a) its session (arguably long period of time)
> has
> > > > timed
> > > > >> > out,
> > > > >> > > > and b) this admin request explicitly state that it is no
> > longer
> > > > >> part of
> > > > >> > > the
> > > > >> > > > group. As for execution I'm fine with keeping it as a future
> > > work
> > > > of
> > > > >> > this
> > > > >> > > > KIP if you'd like to make its scope smaller.
> > > > >> > > >
> > > > >> > > > Following are minor comments:
> > > > >> > > >
> > > > >> > > > 5. I'm not sure if we need to include "member.name" as part
> > of
> > > > the
> > > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > > memberId
> > > > >> plus
> > > > >> > > the
> > > > >> > > > generation number should be sufficient for fencing even with
> > > > static
> > > > >> > > > members.
> > > > >> > > >
> > > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid
> of
> > > the
> > > > "
> > > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > Guozhang
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <
> lindong28@gmail.com
> > >
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Hey Boyang,
> > > > >> > > > >
> > > > >> > > > > Thanks for the proposal! This is very useful. I have some
> > > > comments
> > > > >> > > below:
> > > > >> > > > >
> > > > >> > > > > 1) The motivation currently explicitly states that the
> goal
> > is
> > > > to
> > > > >> > > improve
> > > > >> > > > > performance for heavy state application. It seems that the
> > > > >> motivation
> > > > >> > > can
> > > > >> > > > > be stronger with the following use-case. Currently for
> > > > MirrorMaker
> > > > >> > > > cluster
> > > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> > time
> > > to
> > > > >> > > rolling
> > > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > > process
> > > > >> > restart
> > > > >> > > > > will trigger a rebalance which currently pause the
> > consumption
> > > > of
> > > > >> the
> > > > >> > > all
> > > > >> > > > > partitions of the MirrorMaker cluster. With the change
> > stated
> > > in
> > > > >> this
> > > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > > specified
> > > > >> > > timeout
> > > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > > rebalance
> > > > >> > (e.g.
> > > > >> > > > for
> > > > >> > > > > leader restart) for the entire rolling bounce, which will
> > > > >> > significantly
> > > > >> > > > > improves the availability of the MirrorMaker pipeline. In
> my
> > > > >> opinion,
> > > > >> > > the
> > > > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance
> if
> > > the
> > > > >> > > consumer
> > > > >> > > > > process can be restarted within soon, which helps
> > performance
> > > > >> even if
> > > > >> > > > > overhead of state shuffling for a given process is small.
> > > > >> > > > >
> > > > >> > > > > 2) In order to simplify the KIP reading, can you follow
> the
> > > > >> writeup
> > > > >> > > style
> > > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> > such
> > > as
> > > > >> new
> > > > >> > > > > configs (e.g. registration timeout), new request/response,
> > new
> > > > >> > > > AdminClient
> > > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > > Currently
> > > > >> some
> > > > >> > > of
> > > > >> > > > > these are specified in the Proposed Change section which
> > makes
> > > > it
> > > > >> a
> > > > >> > bit
> > > > >> > > > > inconvenient to understand the new interface that will be
> > > > exposed
> > > > >> to
> > > > >> > > > user.
> > > > >> > > > > Explanation of the current two-phase rebalance protocol
> > > probably
> > > > >> can
> > > > >> > be
> > > > >> > > > > moved out of public interface section.
> > > > >> > > > >
> > > > >> > > > > 3) There are currently two version of JoinGroupRequest in
> > the
> > > > KIP
> > > > >> and
> > > > >> > > > only
> > > > >> > > > > one of them has field memberId. This seems confusing.
> > > > >> > > > >
> > > > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > > > >> rebalance
> > > > >> > > could
> > > > >> > > > > be helpful here, but we will make a call once we finished
> > the
> > > > >> major
> > > > >> > > > > implementation". So this seems to be still an open
> question
> > in
> > > > the
> > > > >> > > > current
> > > > >> > > > > design. We probably want to agree on this before voting
> for
> > > the
> > > > >> KIP.
> > > > >> > > > >
> > > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> > consumer.
> > > > Can
> > > > >> > you
> > > > >> > > > > specify the name of the config key and the default config
> > > value?
> > > > >> > > Possible
> > > > >> > > > > default values include empty string or null (similar to
> > > > >> > transaction.id
> > > > >> > > > in
> > > > >> > > > > producer config).
> > > > >> > > > >
> > > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > > persist
> > > > >> > member
> > > > >> > > > > name map, currently if consumer coordinator broker goes
> > > offline,
> > > > >> > > > rebalance
> > > > >> > > > > is triggered and consumers will try connect to the new
> > > > >> coordinator.
> > > > >> > If
> > > > >> > > > > these consumers can connect to the new coordinator within
> > > > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> > > that
> > > > >> > broker
> > > > >> > > > can
> > > > >> > > > > use a deterministic algorithm to determine the partition
> ->
> > > > >> > member_name
> > > > >> > > > > mapping, each consumer should get assigned the same set of
> > > > >> partitions
> > > > >> > > > > without requiring state shuffling. So it is not clear
> > whether
> > > we
> > > > >> > have a
> > > > >> > > > > strong use-case for this new logic. Can you help clarify
> > what
> > > is
> > > > >> the
> > > > >> > > > > benefit of using topic "static_member_map" to persist
> member
> > > > name
> > > > >> > map?
> > > > >> > > > >
> > > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > > config,
> > > > >> it is
> > > > >> > > > > mentioned that "we are using expansion timeout to replace
> > > > >> rebalance
> > > > >> > > > > timeout, which is configured by max.poll.intervals from
> > client
> > > > >> side,
> > > > >> > > and
> > > > >> > > > > using registration timeout to replace session timeout".
> > > > Currently
> > > > >> the
> > > > >> > > > > default max.poll.interval.ms is configured to be 5
> minutes
> > > and
> > > > >> there
> > > > >> > > > will
> > > > >> > > > > be only one rebalance if all new consumers can join
> within 5
> > > > >> minutes.
> > > > >> > > So
> > > > >> > > > it
> > > > >> > > > > is not clear whether we have a strong use-case for this
> new
> > > > >> config.
> > > > >> > Can
> > > > >> > > > you
> > > > >> > > > > explain what is the benefit of introducing this new
> config?
> > > > >> > > > >
> > > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > > version
> > > > >> of
> > > > >> > > > > protocol, we will also increase the join group request
> > version
> > > > to
> > > > >> v4
> > > > >> > > when
> > > > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> > > latest
> > > > >> (<
> > > > >> > > v4),
> > > > >> > > > > the join group request shall be downgraded to v3 without
> > > setting
> > > > >> the
> > > > >> > > > member
> > > > >> > > > > Id". It is probably simpler to just say that this feature
> is
> > > > >> enabled
> > > > >> > if
> > > > >> > > > > JoinGroupRequest V4 is supported on both client and broker
> > and
> > > > >> > > > MEMBER_NAME
> > > > >> > > > > is configured with non-empty string.
> > > > >> > > > >
> > > > >> > > > > 9) It is mentioned that broker may return
> > > > >> NO_STATIC_MEMBER_INFO_SET
> > > > >> > > error
> > > > >> > > > > in OffsetCommitResponse for "commit requests under static
> > > > >> > membership".
> > > > >> > > > Can
> > > > >> > > > > you clarify how broker determines whether the commit
> request
> > > is
> > > > >> under
> > > > >> > > > > static membership?
> > > > >> > > > >
> > > > >> > > > > Thanks,
> > > > >> > > > > Dong
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > --
> > > > >> > > > -- Guozhang
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > --
> > > > >> > -Regards,
> > > > >> > Mayuresh R. Gharat
> > > > >> > (862) 250-7125
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -- Guozhang
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Guozhang and Mayuresh for the follow up! Answers are listed below.


>  5. Regarding "So in summary, *the member will only be removed due to
> session timeout*. We shall remove it from both in-memory static member name
> mapping and member list." If the rebalance is invoked manually using the
> the admin apis, how long should the group coordinator wait for the members
> of the group to send a JoinGroupRequest for participating in the rebalance?
> How is a lagging consumer handled?

Great question. Let's use c1~c4 example here:

  1.  Consumer c1, c2, c3, c4 in stable state
  2.  c4 goes down and we detect this issue before session timeout through client monitoring. Initiate a ConsumerRebalanceRequest.
  3.  A rebalance will be kicking off, and after rebalance timeout we shall keep the same assignment for c1~4, if the session timeout for c4 hasn't reached
  4.  Group back to stable with c1~4 (although c4 is actually offline)
  5.  c4 session timeout finally reached: another rebalance triggered.

For step 3, if session timeout triggered within rebalance timeout, only c1~3 will be participating in the rebalance. This is what we mean by saying "rebalance
timeout shall not remove current members, only session timeout will do."
As you could see this is not an ideal scenario: we trigger extra rebalance at step 5. In my reply to Guozhang I'm asking whether we should still use LeaveGroupRequest for static members to send a signal to broker saying "I'm currently offline", and when we send ConsumerRebalanceRequest to broker, we will actually kick off c4 because it says it's offline already, saving one or multiple additional rebalances later. This way the ConsumerRebalanceRequest will be more effective in making correct judgement on the group status since we have more feedback from client side.

> - When we say that we would use invokeConsumerRebalance(groupId) to down
> scale, with the example in the above question, how will the
> GroupCoordinator know that c4 should be kicked out of the group since we
> are trying to invoke rebalance proactively without waiting for c4's session
> time out to expire. Should there be a way of telling the GroupCoordinator
> that consumer c4 has been kicked out of the groupId = "GroupA"?
Previous proposal should be suffice to answer this question 😊

- Also it looks like the statement "If the `member.id` uses
> UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> the one within current map, if `group.member.name` is known. Also once we
> are done with KIP-394
> <
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=lEv9SuBZgATGhOSe5zUr%2Fqudycoh%2FwfdM%2FhPH5Hp1N4%3D&amp;reserved=0
> >,
> all the join group requests are requiring `member.id` to physically enter
> the consumer group. This way the latest joined " is incomplete. Can you
> take a look at this?
> Also when we say "all the join group requests are requiring `member.id` to
> physically enter the consumer group." because a newly started consumer will
> not have a "member.id", I assume you mean, once the GroupCoordinator
> assigns a member.id to the newly started consumer, it has to use it for
> any
> future JoinGroupRequests. Is my understanding correct?
>
Thanks for catching it! And yes, we shall use one extra round-trip between consumer
and broker to inform the new member id allocation.

Next is the replies to Guozhang's comment:
2) I once have a discussion about the LeaveGroupRequest for static members,
and the reason for not having it for static members is that we'd need to
make it a configurable behavior as well (i.e. the likelihood that a static
member may shutdown but come back later may be even larger than the
likelihood that a shutdown static member would not come back), and when a
shutdown is complete the instance cannot tell whether or not it will come
back by itself. And hence letting a third party (think: admin used by K8s
plugins) issuing a request to indicate static member changes would be more
plausible.

I think having an optional list of all the static members that are still in
the group, rather than the members to be removed since the latter looks a
bit less flexible to me, in the request is a good idea (remember we allow a
group to have both static and dynamic members at the same time, so when
receiving the request, we will only do the diff and add / remove the static
members directly only, while still let the dynamic members to try to
re-join the group with the rebalance timeout).
I'm also in favor of storing all the in-group static members. In fact we could reuse
the static membership mapping to store this information. Do you think
that we should let static member send leave group request to indicate their status of "leaving",
and use ConsumerRebalanceRequest to trigger rebalance without them? I'm suggesting we should
remove those members when kicking off rebalance since we are shutting them down already.

3) personally I favor "ids" over "names" :) Since we already have some
"ids" and hence it sounds more consistent, plus on the producer side we
have a `transactional.id` whose semantics is a bit similar to this one,
i.e. for unique distinguishment of a client which may comes and goes but
need to be persist over multiple "instance life-times".
Sure we have enough votes for ids 😊I will finalize the name to `group.instance.id`, does that
sound good?

Best,
Boyang
________________________________
From: Guozhang Wang <wa...@gmail.com>
Sent: Wednesday, November 28, 2018 4:51 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Regarding Jason's question and Boyang's responses:

2) I once have a discussion about the LeaveGroupRequest for static members,
and the reason for not having it for static members is that we'd need to
make it a configurable behavior as well (i.e. the likelihood that a static
member may shutdown but come back later may be even larger than the
likelihood that a shutdown static member would not come back), and when a
shutdown is complete the instance cannot tell whether or not it will come
back by itself. And hence letting a third party (think: admin used by K8s
plugins) issuing a request to indicate static member changes would be more
plausible.

I think having an optional list of all the static members that are still in
the group, rather than the members to be removed since the latter looks a
bit less flexible to me, in the request is a good idea (remember we allow a
group to have both static and dynamic members at the same time, so when
receiving the request, we will only do the diff and add / remove the static
members directly only, while still let the dynamic members to try to
re-join the group with the rebalance timeout).

3) personally I favor "ids" over "names" :) Since we already have some
"ids" and hence it sounds more consistent, plus on the producer side we
have a `transactional.id` whose semantics is a bit similar to this one,
i.e. for unique distinguishment of a client which may comes and goes but
need to be persist over multiple "instance life-times".


Guozhang


On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> Hi Boyang,
>
> Thanks for the replies. Please find the follow up queries below.
>
>     5. Regarding "So in summary, *the member will only be removed due to
> session timeout*. We shall remove it from both in-memory static member name
> mapping and member list." If the rebalance is invoked manually using the
> the admin apis, how long should the group coordinator wait for the members
> of the group to send a JoinGroupRequest for participating in the rebalance?
> How is a lagging consumer handled?
> The plan is to disable member kick out when rebalance.timeout is reached,
> so basically we are not "waiting" any
> join group request from existing members; we shall just rebalance base on
> what we currently have within the group
> metadata. Lagging consumer will trigger rebalance later if session timeout
> > rebalance timeout.
>
> >
> Just wanted to understand this better. Lets take an example, say we have a
> > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > Everything is running fine and suddenly C4 host has issues and it goes
> > down. Now we notice that we can still operate with c1, c2, c3 and don't
> > want to wait for
> > c4 to come back up. We use the admin api
> > "invokeConsumerRebalance("GroupA")".
> > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > group again (in there heartBeatResponse) as first step of rebalance.
> > Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> > delayed. At this stage, if we are not "waiting" on any join group
> request,
> > few things can happen :
> >
> >    - c4's partitions are distributed only among c1,c2. c3 maintains its
> >    original assignment. c1, c2 will start processing the newly assigned
> >    partitions.
> >
> > OR
> >
> >    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
> >    processing the newly assigned partitions. c3 gets to know about the
> newly
> >    assigned partitions later when it sends the JoinGroupRequest (which
> was
> >    delayed).
> >
> > OR
> >
> >    - Will the rebalance do a complete reassignment, where c1, c2, c3 have
> >    to give up there partitions and all the partitions belonging to c1,
> c2, c3,
> >    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
> >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> revoke
> >    there partitions and rejoin the group.
> >
> > This is as per my understanding of how the KIP would work without
> changing
> > the underlying group coordination workflow. Please correct me if I
> > misunderstood something here.
> >
>
>
> - When we say that we would use invokeConsumerRebalance(groupId) to down
> scale, with the example in the above question, how will the
> GroupCoordinator know that c4 should be kicked out of the group since we
> are trying to invoke rebalance proactively without waiting for c4's session
> time out to expire. Should there be a way of telling the GroupCoordinator
> that consumer c4 has been kicked out of the groupId = "GroupA"?
>
> - Also it looks like the statement "If the `member.id` uses
> UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> the one within current map, if `group.member.name` is known. Also once we
> are done with KIP-394
> <
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember.id%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=lEv9SuBZgATGhOSe5zUr%2Fqudycoh%2FwfdM%2FhPH5Hp1N4%3D&amp;reserved=0
> >,
> all the join group requests are requiring `member.id` to physically enter
> the consumer group. This way the latest joined " is incomplete. Can you
> take a look at this?
> Also when we say "all the join group requests are requiring `member.id` to
> physically enter the consumer group." because a newly started consumer will
> not have a "member.id", I assume you mean, once the GroupCoordinator
> assigns a member.id to the newly started consumer, it has to use it for
> any
> future JoinGroupRequests. Is my understanding correct?
>
>
> Thanks,
>
> Mayuresh
>
> On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer both
> > in this reply.
> >
> >
> > >    1. Do you intend to have member.id is a static config like
> > member.name
> > >    after KIP-345 and KIP-394?
> >
> > No, we shall only rely on broker to allocate member.id for the consumer
> > instances. FYI, I already
> >
> > started the discussion thread for KIP-394 😊
> >
> > >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> > in
> > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config
> is
> > > set,
> > >    we will put it in the initial join group request to identify itself
> > as a
> > >    static member (static membership); otherwise, we will still send
> > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > (dynamic
> > >    membership)."
> > >       - What is the value of member_id sent in the first
> JoinGroupRequest
> > >       when member_name is set (using static rebalance)? Is it
> > > UNKNOW_MEMBER_ID?
> >
> > Yes, we could only use unknown member id. Actually this part of the
> > proposal is outdated,
> >
> > let me do another audit of the whole doc. Basically, it is currently
> > impossible to send `member.id`
> >
> > when consumer restarted. Sorry for the confusions!
> >
> > >    3. Regarding "we are requiring member.id (if not unknown) to match
> > the
> > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> > case
> > >    that if we could have members with the same `member.name` (for
> > example
> > >    mis-configured instances with a valid member.id but added a used
> > member
> > >    name on runtime). When member name has duplicates, we could refuse
> > join
> > >    request from members with an outdated `member.id` (since we update
> > the
> > >    mapping upon each join group request). In an edge case where the
> > client
> > >    hits this exception in the response, it is suggesting that some
> other
> > >    consumer takes its spot."
> > >       - The part of "some other consumer takes the spot" would be
> > >       intentional, right? Also when you say " The edge case that if we
> > >       could have members with the same `member.name` (for example
> > >       mis-configured instances *with a valid member.id <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > >
> > > *but
> > >       added a used member name on runtime).", what do you mean by
> *valid
> > >       member id* here? Does it mean that there exist a mapping of
> > >       member.name to member.id like *MemberA -> id1* on the
> > >       GroupCoordinator and this consumer is trying to join with *
> > > member.name
> > >       <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> >
> > = MemberB and member.id <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> >
> > =
> > > id1 *
> > >       ?
> >
> > I would take Jason's advice that each time we have unknown member joining
> > the group, the broker will
> >
> > always assign a new and unique id to track its identity. In this way,
> > consumer with duplicate member name
> >
> > will be fenced.
> >
> > >    4. Depending on your explanation for point 2 and the point 3 above
> > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > >    member_name but unknown member_id, if the consumer sends
> > > "UNKNOW_MEMBER_ID"
> > >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > > give it
> > >    a member_id, is the consumer suppose to remember member_id for
> > >    joinGroupRequests? If yes, how are restarts handled?
> >
> > Like explained above, we shall not materialize the member.id. Instead we
> > need to rely on broker to allocate
> >
> > a unique id for consumer just like what we have now.
> >
> > >    5. Regarding "So in summary, *the member will only be removed due to
> > >    session timeout*. We shall remove it from both in-memory static
> member
> > >    name mapping and member list."
> > >       - If the rebalance is invoked manually using the the admin apis,
> > how
> > >       long should the group coordinator wait for the members of the
> > > group to send
> > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > lagging
> > >       consumer handled?
> >
> > The plan is to disable member kick out when rebalance.timeout is reached,
> > so basically we are not "waiting" any
> >
> > join group request from existing members; we shall just rebalance base on
> > what we currently have within the group
> >
> > metadata. Lagging consumer will trigger rebalance later if session
> timeout
> > > rebalance timeout.
> >
> > >    6. Another detail to take care is that we need to automatically take
> > the
> > >    hash of group id so that we know which broker to send this request
> to.
> > >       - I assume this should be same as the way we find the
> coordinator,
> > >       today right? If yes, should we specify it in the KIP ?
> >
> > Yep, it is. Add FindCoordinatorRequest logic to the script.
> >
> > >    7. Are there any specific failure scenarios when you say "other
> > >    potential failure cases."? It would be good to mention them
> > explicitly,
> > > if
> > >    you think there are any.
> >
> > Nah, I'm gonna remove it because it seems causing more confusion than
> > making my assumption clear, which is
> >
> > "there could be other failure cases that I can't enumerate now" 😊
> >
> > >    8. It would be good to have a rollback plan as you have for roll
> > forward
> > >    in the KIP.
> >
> > Great suggestion! Added a simple rollback plan.
> >
> >
> > Next is answering Jason's suggestions:
> >
> > 1. This may be the same thing that Mayuresh is asking about. I think the
> > suggestion in the KIP is that if a consumer sends JoinGroup with a member
> > name, but no member id, then we will return the current member id
> > associated with that name. It seems in this case that we wouldn't be able
> > to protect from having two consumers active with the same configured
> > member.name? For example, imagine that we had a consumer with
> member.name
> > =A
> > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > instance starts up with member.name=A. If it is also assigned member.id
> =1,
> > then how can we detect the zombie if it comes back to life? Both
> instances
> > will have the same member.id.
> >
> > The goal is to avoid a rebalance on a rolling restart, but we still need
> to
> > fence previous members. I am wondering if we can generate a new
> member.id
> > every time we receive a request from a static member with an unknown
> member
> > id. If the old instance with the same member.name attempts any
> operation,
> > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > subscription of the new instance hasn't changed, then we can skip the
> > rebalance and return the current assignment without forcing a rebalance.
> >
> > The trick to making this work is in the error handling of the zombie
> > consumer. If the zombie simply resets its member.id and rejoins to get a
> > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> > fencing the new member. We want to avoid this. There needs to be an
> > expectation for static members that the member.id of a static member
> will
> > not be changed except when a new member with the same member.name joins
> > the
> > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> > with static member names.
> >
> > Yep, I like this idea! Keep giving out refresh member.id when facing
> > anonymous request will definitely
> >
> > prevent processing bug due to duplicate consumers, however I don't think
> I
> > fully understand the 3rd paragraph where
> >
> > you mentioned  "There needs to be an expectation for static members that
> > the member.id of a static member will
> >
> > not be changed except when a new member with the same member.name joins
> > the group. "  How do you plan
> > to know whether this member is new member or old member? I feel even with
> > zombie consumer takes the ownership,
> > it should be detected very quickly (as MISMATCH_ID exception trigger
> > original consumer instance dies)
> > and end user will start to fix it right away. Is there any similar logic
> > we applied in fencing duplicate `transaction.id`?
> >
> > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far
> as
> > I understand it, it is used for scaling down a consumer group and somehow
> > bypasses normal session timeout expiration. I am wondering how critical
> > this piece is and whether we can leave it for future work. If not, then
> it
> > would be helpful to elaborate on its implementation. How would the
> > coordinator know which members to kick out of the group?
> >
> > This API is needed when we need to immediately trigger rebalance instead
> > of waiting session timeout
> >
> > or rebalance timeout (Emergent scale up/down). It is very necessary to
> > have it for
> >
> > management purpose because user could choose when to trigger rebalance
> > pretty freely,
> >
> > gaining more client side control.
> >
> > In the meanwhile I see your point that we need to actually have the
> > ability to kick out members that we plan
> >
> > to scale down fast (as rebalance timeout no longer kicks any offline
> > member out of the group), I will think of adding an optional
> >
> > list of members that are ready to be removed.
> >
> > Another idea is to let static member send `LeaveGroupRequest` when they
> > are going offline (either scale down or bouncing),
> >
> > and broker will cache this information as "OfflineMembers" without
> > triggering rebalance. When handling ConsumerRebalanceRequest broker will
> >
> > kick the static members that are currently offline and trigger rebalance
> > immediately. How does this plan sound?
> >
> > 3. I've been holding back on mentioning this, but I think we should
> > reconsider the name `member.name`. I think we want something that
> suggests
> > its expectation of uniqueness in the group. How about `group.instance.id
> `
> > to go along with `group.id`?
> >
> > Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> > the namespace idea, and
> >
> > since we already use `member.name` in a lot of context, I decide to
> > rename the config to `group.member.name`
> >
> > which should be sufficient for solving all the concerns we have now.
> > Sounds good?
> >
> >
> > Thank you for your great suggestions! Let me know if my reply makes sense
> > her.
> >
> >
> > Best,
> >
> > Boyang
> >
> > ________________________________
> > From: Jason Gustafson <ja...@confluent.io>
> > Sent: Tuesday, November 27, 2018 7:51 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the updates. Looks like we're headed in the right direction
> and
> > clearly the interest that this KIP is receiving shows how strong the
> > motivation is!
> >
> > I have a few questions:
> >
> > 1. This may be the same thing that Mayuresh is asking about. I think the
> > suggestion in the KIP is that if a consumer sends JoinGroup with a member
> > name, but no member id, then we will return the current member id
> > associated with that name. It seems in this case that we wouldn't be able
> > to protect from having two consumers active with the same configured
> > member.name? For example, imagine that we had a consumer with
> member.name
> > =A
> > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > instance starts up with member.name=A. If it is also assigned member.id
> =1,
> > then how can we detect the zombie if it comes back to life? Both
> instances
> > will have the same member.id.
> >
> > The goal is to avoid a rebalance on a rolling restart, but we still need
> to
> > fence previous members. I am wondering if we can generate a new
> member.id
> > every time we receive a request from a static member with an unknown
> member
> > id. If the old instance with the same member.name attempts any
> operation,
> > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > subscription of the new instance hasn't changed, then we can skip the
> > rebalance and return the current assignment without forcing a rebalance.
> >
> > The trick to making this work is in the error handling of the zombie
> > consumer. If the zombie simply resets its member.id and rejoins to get a
> > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> > fencing the new member. We want to avoid this. There needs to be an
> > expectation for static members that the member.id of a static member
> will
> > not be changed except when a new member with the same member.name joins
> > the
> > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> > with static member names.
> >
> > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far
> as
> > I understand it, it is used for scaling down a consumer group and somehow
> > bypasses normal session timeout expiration. I am wondering how critical
> > this piece is and whether we can leave it for future work. If not, then
> it
> > would be helpful to elaborate on its implementation. How would the
> > coordinator know which members to kick out of the group?
> >
> > 3. I've been holding back on mentioning this, but I think we should
> > reconsider the name `member.name`. I think we want something that
> suggests
> > its expectation of uniqueness in the group. How about `group.instance.id
> `
> > to go along with `group.id`?
> >
> > Thanks,
> > Jason
> >
> >
> >
> > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > gharatmayuresh15@gmail.com>
> > wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks a lot for replying to all the queries and discussions here, so
> > > patiently.
> > > Really appreciate it.
> > >
> > > Had a few questions and suggestions after rereading the current version
> > of
> > > the KIP :
> > >
> > >
> > >    1. Do you intend to have member.id is a static config like
> > member.name
> > >    after KIP-345 and KIP-394?
> > >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> > in
> > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config
> is
> > > set,
> > >    we will put it in the initial join group request to identify itself
> > as a
> > >    static member (static membership); otherwise, we will still send
> > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > (dynamic
> > >    membership)."
> > >       - What is the value of member_id sent in the first
> JoinGroupRequest
> > >       when member_name is set (using static rebalance)? Is it
> > > UNKNOW_MEMBER_ID?
> > >    3. Regarding "we are requiring member.id (if not unknown) to match
> > the
> > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> > case
> > >    that if we could have members with the same `member.name` (for
> > example
> > >    mis-configured instances with a valid member.id but added a used
> > member
> > >    name on runtime). When member name has duplicates, we could refuse
> > join
> > >    request from members with an outdated `member.id` (since we update
> > the
> > >    mapping upon each join group request). In an edge case where the
> > client
> > >    hits this exception in the response, it is suggesting that some
> other
> > >    consumer takes its spot."
> > >       - The part of "some other consumer takes the spot" would be
> > >       intentional, right? Also when you say " The edge case that if we
> > >       could have members with the same `member.name` (for example
> > >       mis-configured instances *with a valid member.id <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> > >
> > > *but
> > >       added a used member name on runtime).", what do you mean by
> *valid
> > >       member id* here? Does it mean that there exist a mapping of
> > >       member.name to member.id like *MemberA -> id1* on the
> > >       GroupCoordinator and this consumer is trying to join with *
> > > member.name
> > >       <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> >
> > = MemberB and member.id <
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=gamxKNXR4pbq5e1DQRbsO%2Fyc6ykDqwcMR%2BJyeFtvYC0%3D&amp;reserved=0
> >
> > =
> > > id1 *
> > >       ?
> > >    4. Depending on your explanation for point 2 and the point 3 above
> > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > >    member_name but unknown member_id, if the consumer sends
> > > "UNKNOW_MEMBER_ID"
> > >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > > give it
> > >    a member_id, is the consumer suppose to remember member_id for
> > >    joinGroupRequests? If yes, how are restarts handled?
> > >    5. Regarding "So in summary, *the member will only be removed due to
> > >    session timeout*. We shall remove it from both in-memory static
> member
> > >    name mapping and member list."
> > >       - If the rebalance is invoked manually using the the admin apis,
> > how
> > >       long should the group coordinator wait for the members of the
> > > group to send
> > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > lagging
> > >       consumer handled?
> > >    6. Another detail to take care is that we need to automatically take
> > the
> > >    hash of group id so that we know which broker to send this request
> to.
> > >       - I assume this should be same as the way we find the
> coordinator,
> > >       today right? If yes, should we specify it in the KIP ?
> > >    7. Are there any specific failure scenarios when you say "other
> > >    potential failure cases."? It would be good to mention them
> > explicitly,
> > > if
> > >    you think there are any.
> > >    8. It would be good to have a rollback plan as you have for roll
> > forward
> > >    in the KIP.
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > >> Hey Dong, thanks for the follow-up here!
> > > >>
> > > >>
> > > >> 1) It is not very clear to the user what is the difference between
> > > >> member.name and client.id as both seems to be used to identify the
> > > >> consumer. I am wondering if it would be more intuitive to name it
> > > >> group.member.name (preferred choice since it matches the current
> > > group.id
> > > >> config name) or rebalance.member.name to explicitly show that the
> id
> > is
> > > >> solely used for rebalance.
> > > >> Great question. I feel `member.name` is enough to explain itself,
> it
> > > >> seems not very
> > > >> helpful to make the config name longer. Comparing `name` with `id`
> > gives
> > > >> user the
> > > >> impression that they have the control over it with customized rule
> > than
> > > >> library decided.
> > > >>
> > > >> 2) In the interface change section it is said that
> > > >> GroupMaxSessionTimeoutMs
> > > >> will be changed to 30 minutes. It seems to suggest that we will
> change
> > > the
> > > >> default value of this config. It does not seem necessary to increase
> > the
> > > >> time of consumer failure detection when user doesn't use static
> > > >> membership.
> > > >> Also, say static membership is enabled, then this default config
> > change
> > > >> will cause a partition to be unavailable for consumption for 30
> > minutes
> > > if
> > > >> there is hard consumer failure, which seems to be worse experience
> > than
> > > >> having unnecessary rebalance (when this timeout is small),
> > particularly
> > > >> for
> > > >> new users of Kafka. Could you explain more why we should make this
> > > change?
> > > >> We are not changing the default session timeout value. We are just
> > > >> changing the
> > > >> cap we are enforcing on the session timeout max value. So this
> change
> > is
> > > >> not affecting
> > > >> what kind of membership end user is using, and loosing the cap is
> > giving
> > > >> end user
> > > >> more flexibility on trade-off between liveness and stability.
> > > >>
> > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > DUPLICATE_STATIC_MEMBER
> > > >> into one error? It seems that these two errors are currently handled
> > by
> > > >> the
> > > >> consumer in the same way. And we don't also don't expect
> > > >> MEMBER_ID_MISMATCH
> > > >> to happen. Thus it is not clear what is the benefit of having two
> > > errors.
> > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error because
> > with
> > > >> the KIP-394<
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=45N6Tu9PTF3JL1BrS3PDDDMJKvsaO%2BL%2FE5Wdg1omDUI%3D&amp;reserved=0
> > > >> >
> > > >> we will automatically fence all join requests with
> UNKNOWN_MEMBER_ID.
> > > >>
> > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > contains
> > > >> member name which is already in the consumer group, however the
> member
> > > id
> > > >> was missing". After a consumer is restarted, it will send a
> > > >> JoinGroupRequest with an existing memberName (as the coordinator has
> > not
> > > >> expired this member from the memory) and memberId
> > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> persisted
> > > >> across consumer restart in the consumer side). Does it mean that
> > > >> JoinGroupRequest from a newly restarted consumer will always be
> > rejected
> > > >> until the sessionTimeoutMs has passed?
> > > >> Same answer as question 3). This part of the logic shall be removed
> > from
> > > >> the proposal.
> > > >>
> > > >> 5) It seems that we always add two methods to the interface
> > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> and
> > > the
> > > >> other without option. Could this be specified in the interface
> change
> > > >> section?
> > > >> Sounds good! Added both methods.
> > > >>
> > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > trigger
> > > >> rebalance? If so, we probably want to specify the command line tool
> > > >> interface similar to
> > > >>
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=m7oZgEZRkfk3Ts2cVCImoeCjx3fjJ1Tt4mCF50loSjs%3D&amp;reserved=0
> > > >> .
> > > >> Added the script.
> > > >>
> > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > meaning
> > > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > > >> simpler
> > > >> to allows this API to trigger rebalance regardless of whether
> consumer
> > > is
> > > >> configured with memberName.
> > > >> Sounds good. Right now I feel for both static and dynamic membership
> > it
> > > is
> > > >> more manageable to introduce the consumer rebalance method through
> > admin
> > > >> client API.
> > > >>
> > > >> 8) It is not very clear how the newly added AdminClient API trigger
> > > >> rebalance. For example, does it send request? Can this be explained
> in
> > > the
> > > >> KIP?
> > > >>
> > > >> Sure, I will add more details to the API.
> > > >>
> > > >>
> > > >> Thanks again for the helpful suggestions!
> > > >>
> > > >>
> > > >> Best,
> > > >> Boyang
> > > >>
> > > >> ________________________________
> > > >> From: Dong Lin <li...@gmail.com>
> > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > >> To: dev
> > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > >> specifying member id
> > > >>
> > > >> Hey Boyang,
> > > >>
> > > >> Thanks for the update! Here are some followup comments:
> > > >>
> > > >> 1) It is not very clear to the user what is the difference between
> > > >> member.name and client.id as both seems to be used to identify the
> > > >> consumer. I am wondering if it would be more intuitive to name it
> > > >> group.member.name (preferred choice since it matches the current
> > > group.id
> > > >> config name) or rebalance.member.name to explicitly show that the
> id
> > is
> > > >> solely used for rebalance.
> > > >>
> > > >> 2) In the interface change section it is said that
> > > >> GroupMaxSessionTimeoutMs
> > > >> will be changed to 30 minutes. It seems to suggest that we will
> change
> > > the
> > > >> default value of this config. It does not seem necessary to increase
> > the
> > > >> time of consumer failure detection when user doesn't use static
> > > >> membership.
> > > >> Also, say static membership is enabled, then this default config
> > change
> > > >> will cause a partition to be unavailable for consumption for 30
> > minutes
> > > if
> > > >> there is hard consumer failure, which seems to be worse experience
> > than
> > > >> having unnecessary rebalance (when this timeout is small),
> > particularly
> > > >> for
> > > >> new users of Kafka. Could you explain more why we should make this
> > > change?
> > > >>
> > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > DUPLICATE_STATIC_MEMBER
> > > >> into one error? It seems that these two errors are currently handled
> > by
> > > >> the
> > > >> consumer in the same way. And we don't also don't expect
> > > >> MEMBER_ID_MISMATCH
> > > >> to happen. Thus it is not clear what is the benefit of having two
> > > errors.
> > > >>
> > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > contains
> > > >> member name which is already in the consumer group, however the
> member
> > > id
> > > >> was missing". After a consumer is restarted, it will send a
> > > >> JoinGroupRequest with an existing memberName (as the coordinator has
> > not
> > > >> expired this member from the memory) and memberId
> > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> persisted
> > > >> across consumer restart in the consumer side). Does it mean that
> > > >> JoinGroupRequest from a newly restarted consumer will always be
> > rejected
> > > >> until the sessionTimeoutMs has passed?
> > > >>
> > > >> 5) It seems that we always add two methods to the interface
> > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> and
> > > the
> > > >> other without option. Could this be specified in the interface
> change
> > > >> section?
> > > >>
> > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > trigger
> > > >> rebalance? If so, we probably want to specify the command line tool
> > > >> interface similar to
> > > >>
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=m7oZgEZRkfk3Ts2cVCImoeCjx3fjJ1Tt4mCF50loSjs%3D&amp;reserved=0
> > > >> .
> > > >>
> > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > meaning
> > > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > > >> simpler
> > > >> to allows this API to trigger rebalance regardless of whether
> consumer
> > > is
> > > >> configured with memberName.
> > > >>
> > > >> 8) It is not very clear how the newly added AdminClient API trigger
> > > >> rebalance. For example, does it send request? Can this be explained
> in
> > > the
> > > >> KIP?
> > > >>
> > > >> Thanks,
> > > >> Dong
> > > >>
> > > >>
> > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > >>
> > > >> > Hey Mayuresh,
> > > >> >
> > > >> >
> > > >> > thanks for your feedbacks! I will try do another checklist here.
> > > >> >
> > > >> >
> > > >> > > By this you mean, even if the application has not called
> > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > sending
> > > >> the
> > > >> > > LeaveGroup request, right?
> > > >> >
> > > >> > Yep it's true, we will prevent client from sending leave group
> > request
> > > >> > when they are set with `member.name`.
> > > >> >
> > > >> >
> > > >> > > When is the member.name removed from this map?
> > > >> > Good question, we will only kick off member due to session timeout
> > > >> within
> > > >> > static membership. Let me update the KIP to clearly assert that.
> > > >> >
> > > >> > > How is this case (missing member id) handled on the client side?
> > > What
> > > >> is
> > > >> > the application that
> > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > >> > I have extended the two exceptions within join group response V4.
> > > >> > Basically I define both corresponding actions to be immediate
> > failing
> > > >> > client application, because so far it is unknown what kind of
> client
> > > >> issue
> > > >> > could trigger them. After the first version, we will keep enhance
> > the
> > > >> error
> > > >> > handling logic!
> > > >> >
> > > >> > > This would mean that it might take more time to detect unowned
> > topic
> > > >> > > partitions and may cause delay for applications that perform
> data
> > > >> > mirroring
> > > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > > make
> > > >> > here
> > > >> > > as listed below separately.
> > > >> > The goal of extending session timeout cap is for users with good
> > > client
> > > >> > side monitoring tools that could auto-heal the dead consumers very
> > > >> fast. So
> > > >> > it is optional (and personal) to extend session timeout to a
> > > reasonable
> > > >> > number with different client scenarios.
> > > >> >
> > > >> > > you meant remove unjoined members of the group, right ?
> > > >> > Yep, there is a typo. Thanks for catching this!
> > > >> >
> > > >> > > What do you mean by " Internally we would optimize this logic by
> > > >> having
> > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > stage,
> > > >> > > without removing non-responsive members immediately." There
> would
> > > not
> > > >> be
> > > >> > a
> > > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > > later,
> > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > >> > No, there won't be. We want to limit the rebalance timeout
> > > functionality
> > > >> > to only use as a timer to
> > > >> > end prepare rebalance stage. This way, late joining static members
> > > will
> > > >> > not trigger further rebalance
> > > >> > as long as they are within session timeout. I added your highlight
> > to
> > > >> the
> > > >> > KIP!
> > > >> >
> > > >> > > The KIP talks about scale up scenario but its not quite clear
> how
> > we
> > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > adding
> > > >> > status
> > > >> > > "learner" ?. Can you shed more light on how this is handled in
> the
> > > >> KIP,
> > > >> > if
> > > >> > > its handled?
> > > >> > Updated the KIP: we shall not cover scale up case in 345, because
> we
> > > >> > believe client side could
> > > >> > better handle this logic.
> > > >> >
> > > >> > > I think Jason had brought this up earlier about having a way to
> > say
> > > >> how
> > > >> > > many members/consumer hosts are you choosing to be in the
> consumer
> > > >> group.
> > > >> > > If we can do this, then in case of mirroring applications we can
> > do
> > > >> this
> > > >> > :
> > > >> > > Lets say we have a mirroring application that consumes from
> Kafka
> > > >> cluster
> > > >> > > A and produces to Kafka cluster B.
> > > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > > >> service
> > > >> > > providers can set a mirroring group saying that it will take,
> for
> > > >> example
> > > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > > >> latency
> > > >> > > for mirroring and can have additional 10 consumer hosts as spare
> > in
> > > >> the
> > > >> > > same group.
> > > >> > > So when the first 300 members/consumers to join the group will
> > start
> > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > >> > > The remaining 10 consumer members can sit idle.
> > > >> > > The moment one of the consumer (for example: consumer number 54)
> > > from
> > > >> the
> > > >> > > first 300 members go out of the group (crossed session timeout),
> > it
> > > >> (the
> > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > >> consumer
> > > >> > > member 54 to one of the spare hosts.
> > > >> > > Once the consumer member 54 comes back up, it can start as
> being a
> > > >> part
> > > >> > of
> > > >> > > the spare pool.
> > > >> > > This enables us to have lower session timeouts and low latency
> > > >> mirroring,
> > > >> > > in cases where the service providers are OK with having spare
> > hosts.
> > > >> > > This would mean that we would tolerate n consumer members
> leaving
> > > and
> > > >> > > rejoining the group and still provide low latency as long as n
> <=
> > > >> number
> > > >> > of
> > > >> > > spare consumers.
> > > >> > > If there are no spare host available, we can get back to the
> idea
> > as
> > > >> > > described in the KIP.
> > > >> > Great idea! In fact on top of static membership we could later
> > > introduce
> > > >> > APIs to set hard-coded
> > > >> > client ids to the group and replace the dead host, or as you
> > proposed
> > > to
> > > >> > define spare host as
> > > >> > what I understood as hot backup. I will put both Jason and your
> > > >> > suggestions into a separate section
> > > >> > called "Future works". Note that this spare host idea may be also
> > > >> solvable
> > > >> > through rebalance protocol
> > > >> > IMO.
> > > >> >
> > > >> > Thank you again for the great feedback!
> > > >> >
> > > >> > Boyang
> > > >> > ________________________________
> > > >> > From: Boyang Chen <bc...@outlook.com>
> > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > >> > specifying member id
> > > >> >
> > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> email
> > > on
> > > >> my
> > > >> > thread, so I will just do a checklist here!
> > > >> >
> > > >> >
> > > >> > 1) The motivation currently explicitly states that the goal is to
> > > >> improve
> > > >> >
> > > >> > performance for heavy state application. It seems that the
> > motivation
> > > >> can
> > > >> >
> > > >> > be stronger with the following use-case. Currently for MirrorMaker
> > > >> cluster
> > > >> >
> > > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > rolling
> > > >> >
> > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > > restart
> > > >> >
> > > >> > will trigger a rebalance which currently pause the consumption of
> > the
> > > >> all
> > > >> >
> > > >> > partitions of the MirrorMaker cluster. With the change stated in
> > this
> > > >> >
> > > >> > patch, as long as a MirrorMaker can restart within the specified
> > > timeout
> > > >> >
> > > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> > (e.g.
> > > >> for
> > > >> >
> > > >> > leader restart) for the entire rolling bounce, which will
> > > significantly
> > > >> >
> > > >> > improves the availability of the MirrorMaker pipeline. In my
> > opinion,
> > > >> the
> > > >> >
> > > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > >> consumer
> > > >> >
> > > >> > process can be restarted within soon, which helps performance even
> > if
> > > >> >
> > > >> > overhead of state shuffling for a given process is small.
> > > >> >
> > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > making
> > > >> the
> > > >> > motivation more solid!
> > > >> >
> > > >> > 2) In order to simplify the KIP reading, can you follow the
> writeup
> > > >> style
> > > >> > of other KIP (e.g. KIP-98) and list the interface change such as
> new
> > > >> > configs (e.g. registration timeout), new request/response, new
> > > >> AdminClient
> > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> > some
> > > of
> > > >> > these are specified in the Proposed Change section which makes it
> a
> > > bit
> > > >> > inconvenient to understand the new interface that will be exposed
> to
> > > >> user.
> > > >> > Explanation of the current two-phase rebalance protocol probably
> can
> > > be
> > > >> > moved out of public interface section.
> > > >> > This is a great suggestion! I just consolidated all the public API
> > > >> > changes, and the whole KIP
> > > >> > looks much more organized!
> > > >> >
> > > >> > 3) There are currently two version of JoinGroupRequest in the KIP
> > and
> > > >> only
> > > >> > one of them has field memberId. This seems confusing.
> > > >> > Yep, I already found this issue and fixed it.
> > > >> >
> > > >> > 4) It is mentioned in the KIP that "An admin API to force
> rebalance
> > > >> could
> > > >> > be helpful here, but we will make a call once we finished the
> major
> > > >> > implementation". So this seems to be still an open question in the
> > > >> current
> > > >> > design. We probably want to agree on this before voting for the
> KIP.
> > > >> > We have finalized the idea that this API is needed.
> > > >> >
> > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> > you
> > > >> > specify the name of the config key and the default config value?
> > > >> Possible
> > > >> > default values include empty string or null (similar to
> > > transaction.id<
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ecs5ZNM3ZUKB7W3u6yzmihcZrTw%2BOHnF16oRPLOlYB8%3D&amp;reserved=0
> > > >> >
> > > >> > in
> > > >> > producer config).
> > > >> > I have defined the `member.name` in "New configuration" section.
> > > >> >
> > > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > > member
> > > >> > name map, currently if consumer coordinator broker goes offline,
> > > >> rebalance
> > > >> > is triggered and consumers will try connect to the new
> coordinator.
> > If
> > > >> > these consumers can connect to the new coordinator within
> > > >> > max.poll.interval.ms<
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ufEq5FifjLJYW0N0bKABNETkGnB9PgcYitoVJrYrr%2Bw%3D&amp;reserved=0
> > > >> >
> > > >> > which by default is 5 minutes, given that broker can
> > > >> > use a deterministic algorithm to determine the partition ->
> > > member_name
> > > >> > mapping, each consumer should get assigned the same set of
> > partitions
> > > >> > without requiring state shuffling. So it is not clear whether we
> > have
> > > a
> > > >> > strong use-case for this new logic. Can you help clarify what is
> the
> > > >> > benefit of using topic "static_member_map" to persist member name
> > map?
> > > >> > I have discussed with Guozhang offline, and I believe reusing the
> > > >> current
> > > >> > `_consumer_offsets`
> > > >> > topic is a better and unified solution.
> > > >> >
> > > >> > 7) Regarding the introduction of the expensionTimeoutMs config, it
> > is
> > > >> > mentioned that "we are using expansion timeout to replace
> rebalance
> > > >> > timeout, which is configured by max.poll.intervals from client
> side,
> > > and
> > > >> > using registration timeout to replace session timeout". Currently
> > the
> > > >> > default max.poll.interval.ms<
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=ufEq5FifjLJYW0N0bKABNETkGnB9PgcYitoVJrYrr%2Bw%3D&amp;reserved=0
> > > >> >
> > > >> > is configured to be 5 minutes and there will
> > > >> > be only one rebalance if all new consumers can join within 5
> > minutes.
> > > >> So it
> > > >> > is not clear whether we have a strong use-case for this new
> config.
> > > Can
> > > >> you
> > > >> > explain what is the benefit of introducing this new config?
> > > >> > Previously our goal is to use expansion timeout as a workaround
> for
> > > >> > triggering multiple
> > > >> > rebalances when scaling up members are not joining at the same
> time.
> > > It
> > > >> is
> > > >> > decided to
> > > >> > be addressed by client side protocol change, so we will not
> > introduce
> > > >> > expansion timeout.
> > > >> >
> > > >> > 8) It is mentioned that "To distinguish between previous version
> of
> > > >> > protocol, we will also increase the join group request version to
> v4
> > > >> when
> > > >> > MEMBER_NAME is set" and "If the broker version is not the latest
> (<
> > > v4),
> > > >> > the join group request shall be downgraded to v3 without setting
> the
> > > >> member
> > > >> > Id". It is probably simpler to just say that this feature is
> enabled
> > > if
> > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > >> MEMBER_NAME
> > > >> > is configured with non-empty string.
> > > >> > Yep, addressed this!
> > > >> >
> > > >> > 9) It is mentioned that broker may return
> NO_STATIC_MEMBER_INFO_SET
> > > >> error
> > > >> > in OffsetCommitResponse for "commit requests under static
> > membership".
> > > >> Can
> > > >> > you clarify how broker determines whether the commit request is
> > under
> > > >> > static membership?
> > > >> >
> > > >> > We have agreed that commit request shouldn't be affected by the
> new
> > > >> > membership, thus
> > > >> > removing it here. Thanks for catching this!
> > > >> >
> > > >> > Let me know if you have further suggestions or concerns. Thank you
> > for
> > > >> > your valuable feedback
> > > >> > to help me design the KIP better! (And I will try to address your
> > > >> > feedbacks in next round Mayuresh ??)
> > > >> >
> > > >> > Best,
> > > >> > Boyang
> > > >> > ________________________________
> > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > >> > specifying member id
> > > >> >
> > > >> > Hi Boyang,
> > > >> >
> > > >> > Thanks for updating the KIP. This is a step good direction for
> > > stateful
> > > >> > applications and also mirroring applications whose latency is
> > affected
> > > >> due
> > > >> > to the rebalance issues that we have today.
> > > >> >
> > > >> > I had a few questions on the current version of the KIP :
> > > >> > For the effectiveness of the KIP, consumer with member.name set
> > will
> > > >> *not
> > > >> > send leave group request* when they go offline
> > > >> >
> > > >> > > By this you mean, even if the application has not called
> > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > sending
> > > >> the
> > > >> > > LeaveGroup request, right?
> > > >> > >
> > > >> >
> > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > member.id
> > > }
> > > >> to
> > > >> > track member uniqueness.
> > > >> >
> > > >> > > When is the member.name removed from this map?
> > > >> > >
> > > >> >
> > > >> > Member.id must be set if the *member.name <
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=Cxjeu8HZ%2Fm6QTuCtNrzmT5dEKKYa%2FyWhfgGxSsQD7dk%3D&amp;reserved=0
> > > >> >
> > > >> > *is already
> > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > >> >
> > > >> > > How is this case handled on the client side? What is the
> > application
> > > >> that
> > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > >> > >
> > > >> >
> > > >> > Session timeout is the timeout we will trigger rebalance when a
> > member
> > > >> goes
> > > >> > offline for too long (not sending heartbeat request). To make
> static
> > > >> > membership effective, we should increase the default max session
> > > >> timeout to
> > > >> > 30 min so that end user could config it freely.
> > > >> >
> > > >> > > This would mean that it might take more time to detect unowned
> > topic
> > > >> > > partitions and may cause delay for applications that perform
> data
> > > >> > mirroring
> > > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > > make
> > > >> > here
> > > >> > > as listed below separately.
> > > >> > >
> > > >> >
> > > >> > Currently there is a config called *rebalance timeout* which is
> > > >> configured
> > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > interval
> > > >> is
> > > >> > because consumer could only send request within the call of poll()
> > and
> > > >> we
> > > >> > want to wait sufficient time for the join group request. When
> > reaching
> > > >> > rebalance timeout, the group will move towards completingRebalance
> > > stage
> > > >> > and remove unjoined groups
> > > >> >
> > > >> > > you meant remove unjoined members of the group, right ?
> > > >> > >
> > > >> >
> > > >> > Currently there is a config called *rebalance timeout* which is
> > > >> configured
> > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > interval
> > > >> is
> > > >> > because consumer could only send request within the call of poll()
> > and
> > > >> we
> > > >> > want to wait sufficient time for the join group request. When
> > reaching
> > > >> > rebalance timeout, the group will move towards completingRebalance
> > > stage
> > > >> > and remove unjoined groups. This is actually conflicting with the
> > > >> design of
> > > >> > static membership, because those temporarily unavailable members
> > will
> > > >> > potentially reattempt the join group and trigger extra rebalances.
> > > >> > Internally we would optimize this logic by having rebalance
> timeout
> > > >> only in
> > > >> > charge of stopping prepare rebalance stage, without removing
> > > >> non-responsive
> > > >> > members immediately.
> > > >> >
> > > >> > > What do you mean by " Internally we would optimize this logic by
> > > >> having
> > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > stage,
> > > >> > > without removing non-responsive members immediately." There
> would
> > > not
> > > >> be
> > > >> > a
> > > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > > later,
> > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > >> > >
> > > >> >
> > > >> > Scale Up
> > > >> >
> > > >> > > The KIP talks about scale up scenario but its not quite clear
> how
> > we
> > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > adding
> > > >> > status
> > > >> > > "learner" ?. Can you shed more light on how this is handled in
> the
> > > >> KIP,
> > > >> > if
> > > >> > > its handled?
> > > >> > >
> > > >> >
> > > >> >
> > > >> > *Discussion*
> > > >> > Larger session timeouts causing latency rise for getting data for
> > > >> un-owned
> > > >> > topic partitions :
> > > >> >
> > > >> > > I think Jason had brought this up earlier about having a way to
> > say
> > > >> how
> > > >> > > many members/consumer hosts are you choosing to be in the
> consumer
> > > >> group.
> > > >> > > If we can do this, then in case of mirroring applications we can
> > do
> > > >> this
> > > >> > :
> > > >> > > Lets say we have a mirroring application that consumes from
> Kafka
> > > >> cluster
> > > >> > > A and produces to Kafka cluster B.
> > > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > > >> service
> > > >> > > providers can set a mirroring group saying that it will take,
> for
> > > >> example
> > > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > > >> latency
> > > >> > > for mirroring and can have additional 10 consumer hosts as spare
> > in
> > > >> the
> > > >> > > same group.
> > > >> > > So when the first 300 members/consumers to join the group will
> > start
> > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > >> > > The remaining 10 consumer members can sit idle.
> > > >> > > The moment one of the consumer (for example: consumer number 54)
> > > from
> > > >> the
> > > >> > > first 300 members go out of the group (crossed session timeout),
> > it
> > > >> (the
> > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > >> consumer
> > > >> > > member 54 to one of the spare hosts.
> > > >> > > Once the consumer member 54 comes back up, it can start as
> being a
> > > >> part
> > > >> > of
> > > >> > > the spare pool.
> > > >> > > This enables us to have lower session timeouts and low latency
> > > >> mirroring,
> > > >> > > in cases where the service providers are OK with having spare
> > hosts.
> > > >> > > This would mean that we would tolerate n consumer members
> leaving
> > > and
> > > >> > > rejoining the group and still provide low latency as long as n
> <=
> > > >> number
> > > >> > of
> > > >> > > spare consumers.
> > > >> > > If there are no spare host available, we can get back to the
> idea
> > as
> > > >> > > described in the KIP.
> > > >> > >
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Mayuresh
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > >> > konstantine@confluent.io> wrote:
> > > >> >
> > > >> > > Hi Boyang.
> > > >> > >
> > > >> > > Thanks for preparing this KIP! It is making good progress and
> will
> > > be
> > > >> a
> > > >> > > great improvement for stateful Kafka applications.
> > > >> > >
> > > >> > > Apologies for my late reply, I was away for a while. Lots of
> great
> > > >> > comments
> > > >> > > so far, so I'll probably second most of them in what I suggest
> > below
> > > >> at
> > > >> > > this point.
> > > >> > >
> > > >> > > When I first read the KIP, I wanted to start at the end with
> > > something
> > > >> > that
> > > >> > > wasn't highlighted a lot. That was the topic related to handling
> > > >> > duplicate
> > > >> > > members. I see now that the initial suggestion of handling this
> > > >> situation
> > > >> > > during offset commit has been removed, and I agree with that.
> > Issues
> > > >> > > related to membership seem to be handled better when the member
> > > joins
> > > >> the
> > > >> > > group rather than when it tries to commit offsets. This also
> > > >> simplifies
> > > >> > how
> > > >> > > many request types need to change in order to incorporate the
> new
> > > >> member
> > > >> > > name field.
> > > >> > >
> > > >> > > I also agree with what Jason and Guozhang have said regarding
> > > >> timeouts.
> > > >> > > Although semantically, it's easier to think of every operation
> > > having
> > > >> its
> > > >> > > own timeout, operationally this can become a burden. Thus,
> > > >> consolidation
> > > >> > > seems preferable here. The definition of embedded protocols on
> top
> > > of
> > > >> the
> > > >> > > base group membership protocol for rebalancing gives enough
> > > >> flexibility
> > > >> > to
> > > >> > > address such needs in each client component separately.
> > > >> > >
> > > >> > > Finally, some minor comments:
> > > >> > > In a few places the new/proposed changes are referred to as
> > > "current".
> > > >> > > Which is a bit confusing considering that there is a protocol in
> > > place
> > > >> > > already, and by "current" someone might understand the existing
> > one.
> > > >> I'd
> > > >> > > recommend using new/proposed or equivalent when referring to
> > changes
> > > >> > > introduced with KIP-345 and current/existing or equivalent when
> > > >> referring
> > > >> > > to existing behavior.
> > > >> > >
> > > >> > > There's the following sentence in the "Public Interfaces"
> section:
> > > >> > > "Since for many stateful consumer/stream applications, the state
> > > >> > shuffling
> > > >> > > is more painful than short time partial unavailability."
> > > >> > > However, my understanding is that the changes proposed with
> > KIP-345
> > > >> will
> > > >> > > not exploit any partial availability. A suggestion for dealing
> > with
> > > >> > > temporary imbalances has been made in "Incremental Cooperative
> > > >> > Rebalancing"
> > > >> > > which can work well with KIP-345, but here I don't see proposed
> > > >> changes
> > > >> > > that suggest that some resources (e.g. partitions) will keep
> being
> > > >> used
> > > >> > > while others will not be utilized. Thus, you might want to
> adjust
> > > this
> > > >> > > sentence. Correct me if I'm missing something related to that.
> > > >> > >
> > > >> > > In the rejected alternatives, under point 2) I read "we can copy
> > the
> > > >> > member
> > > >> > > id to the config files". I believe it means to say "member name"
> > > >> unless
> > > >> > I'm
> > > >> > > missing something about reusing member ids. Also below I read:
> "By
> > > >> > allowing
> > > >> > > consumers to optionally specifying a member id" which probably
> > > implies
> > > >> > > "member name" again. In a sense this section highlights a
> > potential
> > > >> > > confusion between member name and member id. I wonder if we
> could
> > > >> come up
> > > >> > > with a better term for the new field. StaticTag, StaticLabel, or
> > > even
> > > >> > > StaticName are some suggestions that could potentially help with
> > > >> > confusion
> > > >> > > between MemberId and MemberName and what corresponds to what.
> But
> > I
> > > >> > > wouldn't like to disrupt the discussion with naming conventions
> > too
> > > >> much
> > > >> > at
> > > >> > > this point. I just mention it here as a thought.
> > > >> > >
> > > >> > > Looking forward to see the final details of this KIP. Great work
> > so
> > > >> far!
> > > >> > >
> > > >> > > Konstantine
> > > >> > >
> > > >> > >
> > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> bchen11@outlook.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Thanks Guozhang for the great summary here, and I have been
> > > >> following
> > > >> > up
> > > >> > > > the action items here.
> > > >> > > >
> > > >> > > >
> > > >> > > >   1.  I already updated the KIP to remove the expansion
> timeout
> > > and
> > > >> > > > registration timeout. Great to see them being addressed in
> > client
> > > >> side!
> > > >> > > >   2.  I double checked the design and I believe that it is ok
> to
> > > >> have
> > > >> > > both
> > > >> > > > static member and dynamic member co-exist in the same group.
> So
> > > the
> > > >> > > upgrade
> > > >> > > > shouldn't be destructive and we are removing the two
> membership
> > > >> > protocol
> > > >> > > > switching APIs.
> > > >> > > >   3.  I only have question about this one. I'm still reading
> the
> > > >> > > KafkaApis
> > > >> > > > code here. Should I just use the same authorization logic for
> > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > suggested,
> > > >> > this
> > > >> > > > feature could be better addressed in a separate KIP because it
> > is
> > > >> > pretty
> > > >> > > > independent. I could start drafting the KIP once the current
> > > >> proposal
> > > >> > is
> > > >> > > > approved.
> > > >> > > >   5.  I believe that we don't need fencing in offset commit
> > > request,
> > > >> > > since
> > > >> > > > duplicate member.name issue could be handled by join group
> > > >> request. We
> > > >> > > > shall reject join group with known member name but no member
> id
> > > >> (which
> > > >> > > > means we already have an active member using this identity).
> > > >> > > >   6.  I agree to remove that internal config once we move
> > forward
> > > >> with
> > > >> > > > static membership. And I already removed the entire section
> from
> > > the
> > > >> > KIP.
> > > >> > > >
> > > >> > > > Let me know if you have other concerns.
> > > >> > > >
> > > >> > > > Best,
> > > >> > > > Boyang
> > > >> > > > ________________________________
> > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > >> > > > To: dev
> > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > >> by
> > > >> > > > specifying member id
> > > >> > > >
> > > >> > > > Hello Boyang,
> > > >> > > >
> > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > appreciate
> > > >> your
> > > >> > > > patience answering to the feedbacks from the community. I'd
> like
> > > to
> > > >> add
> > > >> > > my
> > > >> > > > 2cents here:
> > > >> > > >
> > > >> > > > 1. By introducing another two timeout configs,
> > > registration_timeout
> > > >> and
> > > >> > > > expansion_timeout, we are effectively having four timeout
> > configs:
> > > >> > > session
> > > >> > > > timeout, rebalance timeout (configured as "
> max.poll.interval.ms
> > "
> > > on
> > > >> > > client
> > > >> > > > side), and these two. Interplaying these timeout configs can
> be
> > > >> quite
> > > >> > > hard
> > > >> > > > for users with such complexity, and hence I'm wondering if we
> > can
> > > >> > > simplify
> > > >> > > > the situation with as less possible timeout configs as
> possible.
> > > >> Here
> > > >> > is
> > > >> > > a
> > > >> > > > concrete suggestion I'd like propose:
> > > >> > > >
> > > >> > > > 1.a) Instead of introducing a registration_timeout in addition
> > to
> > > >> the
> > > >> > > > session_timeout for static members, we can just reuse the
> > > >> > session_timeout
> > > >> > > > and ask users to set it to a larger value when they are
> > upgrading
> > > a
> > > >> > > dynamic
> > > >> > > > client to a static client by setting the "member.name" at the
> > > same
> > > >> > time.
> > > >> > > > By
> > > >> > > > default, the broker-side min.session.timeout is 6 seconds and
> > > >> > > > max.session.timeout is 5 minutes, which seems reasonable to me
> > (we
> > > >> can
> > > >> > of
> > > >> > > > course modify this broker config to enlarge the valid interval
> > if
> > > we
> > > >> > want
> > > >> > > > in practice). And then we should also consider removing the
> > > >> condition
> > > >> > for
> > > >> > > > marking a client as failed if the rebalance timeout has
> reached
> > > >> while
> > > >> > the
> > > >> > > > JoinGroup was not received, so that the semantics of
> > > session_timeout
> > > >> > and
> > > >> > > > rebalance_timeout are totally separated: the former is only
> used
> > > to
> > > >> > > > determine if a consumer member of the group should be marked
> as
> > > >> failed
> > > >> > > and
> > > >> > > > kicked out of the group, and the latter is only used to
> > determine
> > > >> the
> > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > phase.
> > > In
> > > >> > > other
> > > >> > > > words if a member did not send the JoinGroup in time of the
> > > >> > > > rebalance_timeout, we still include it in the new generation
> of
> > > the
> > > >> > group
> > > >> > > > and use its old subscription info to send to leader for
> > > assignment.
> > > >> > Later
> > > >> > > > if the member came back with HeartBeat request, we can still
> > > follow
> > > >> the
> > > >> > > > normal path to bring it to the latest generation while
> checking
> > > that
> > > >> > its
> > > >> > > > sent JoinGroup request contains the same subscription info as
> we
> > > >> used
> > > >> > to
> > > >> > > > assign the partitions previously (which should be likely the
> > case
> > > in
> > > >> > > > practice). In addition, we should let static members to not
> send
> > > the
> > > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > > static
> > > >> > > member
> > > >> > > > can only be leaving the group if its session has timed out, OR
> > it
> > > >> has
> > > >> > > been
> > > >> > > > indicated to not exist in the group any more (details below).
> > > >> > > >
> > > >> > > > 1.b) We have a parallel discussion about Incremental
> Cooperative
> > > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> > logic
> > > >> at
> > > >> > the
> > > >> > > > application level, instead of at the protocol level. By doing
> > this
> > > >> we
> > > >> > can
> > > >> > > > also enable a few other optimizations, e.g. at the Streams
> level
> > > to
> > > >> > first
> > > >> > > > build up the state store as standby tasks and then trigger a
> > > second
> > > >> > > > rebalance to actually migrate the active tasks while keeping
> the
> > > >> actual
> > > >> > > > rebalance latency and hence unavailability window to be small
> (
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=eHHZXH6id%2FihibQ9640lWEsjz3VKROW9JfvTni4HO2s%3D&amp;reserved=0
> > > >> > > ).
> > > >> > > > I'd propose we align
> > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > >> > expansion_timeout
> > > >> > > as
> > > >> > > > part of the protocol layer, but only do that at the
> > application's
> > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > > still,
> > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > >> > > > <
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://nam02.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Ccd1e9e2eee0f440987bf08d654aa1dd9%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636789486977454595&amp;sdata=8iZF4R9mdfnEH4RPPa%2ButUWhrBVRYuRVHi98RsbcRPQ%3D&amp;reserved=0
> > > >> > > >*"
> > > >> > > > though as part of this KIP
> > > >> > > > since we have discussed about its limit and think it is
> actually
> > > >> not a
> > > >> > > very
> > > >> > > > good design and could be replaced with client-side logic
> above.
> > > >> > > >
> > > >> > > >
> > > >> > > > 2. I'd like to see your thoughts on the upgrade path for this
> > KIP.
> > > >> More
> > > >> > > > specifically, let's say after we have upgraded broker version
> to
> > > be
> > > >> > able
> > > >> > > to
> > > >> > > > recognize the new versions of JoinGroup request and the admin
> > > >> requests,
> > > >> > > how
> > > >> > > > should we upgrade the clients and enable static groups? On top
> > of
> > > my
> > > >> > head
> > > >> > > > if we do a rolling bounce in which we set the member.name
> > config
> > > as
> > > >> > well
> > > >> > > > as
> > > >> > > > optionally increase the session.timeout config when we bounce
> > each
> > > >> > > > instance, then during this rolling bounces we will have a
> group
> > > >> > contained
> > > >> > > > with both dynamic members and static members. It means that we
> > > >> should
> > > >> > > have
> > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > JoinGroup
> > > >> > > requests
> > > >> > > > from dynamic members), and hence the "member.name" -> "
> > member.id"
> > > >> > > mapping
> > > >> > > > will only be partial at this scenario. Also could you describe
> > if
> > > >> the
> > > >> > > > upgrade to the first version that support this feature would
> > ever
> > > >> get
> > > >> > any
> > > >> > > > benefits, or only the future upgrade path for rolling bounces
> > > could
> > > >> get
> > > >> > > > benefits out of this feature?
> > > >> > > >
> > > >> > > > If that's the case and we will do 1) as suggested above, do we
> > > still
> > > >> > need
> > > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > > >> requests
> > > >> > any
> > > >> > > > more? Seems it is not necessary any more as we will only have
> > the
> > > >> > notion
> > > >> > > of
> > > >> > > > "dynamic or static members" that can co-exist in a group while
> > > >> there no
> > > >> > > > notion of "dynamic or static groups", and hence these two
> > requests
> > > >> are
> > > >> > > not
> > > >> > > > needed anymore.
> > > >> > > >
> > > >> > > >
> > > >> > > > 3. We need to briefly talk about the implications for ACL as
> we
> > > >> > introduce
> > > >> > > > new admin requests that are related to a specific group.id.
> For
> > > >> > example,
> > > >> > > > we
> > > >> > > > need to make sure that whoever created the group or joined the
> > > group
> > > >> > can
> > > >> > > > actually send admin requests for the group, otherwise the
> > > >> application
> > > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > > cluster
> > > >> > every
> > > >> > > > time they want to send any admin requests for their groups
> which
> > > >> would
> > > >> > be
> > > >> > > > an operational nightmare.
> > > >> > > >
> > > >> > > >
> > > >> > > > 4. I like Jason's suggestion of adding an optional field for
> the
> > > >> list
> > > >> > of
> > > >> > > > member names, and I'm wondering if that can be done as part of
> > the
> > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> members,
> > > we
> > > >> > will
> > > >> > > > enforce a rebalance immediately since it indicates that some
> > > static
> > > >> > > member
> > > >> > > > will be officially kicked out of the group and some new static
> > > >> members
> > > >> > > may
> > > >> > > > be added. So back to 1.a) above, a static member can only be
> > > kicked
> > > >> out
> > > >> > > of
> > > >> > > > the group if a) its session (arguably long period of time) has
> > > timed
> > > >> > out,
> > > >> > > > and b) this admin request explicitly state that it is no
> longer
> > > >> part of
> > > >> > > the
> > > >> > > > group. As for execution I'm fine with keeping it as a future
> > work
> > > of
> > > >> > this
> > > >> > > > KIP if you'd like to make its scope smaller.
> > > >> > > >
> > > >> > > > Following are minor comments:
> > > >> > > >
> > > >> > > > 5. I'm not sure if we need to include "member.name" as part
> of
> > > the
> > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > memberId
> > > >> plus
> > > >> > > the
> > > >> > > > generation number should be sufficient for fencing even with
> > > static
> > > >> > > > members.
> > > >> > > >
> > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid of
> > the
> > > "
> > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > >> > > >
> > > >> > > >
> > > >> > > > Guozhang
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <lindong28@gmail.com
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Hey Boyang,
> > > >> > > > >
> > > >> > > > > Thanks for the proposal! This is very useful. I have some
> > > comments
> > > >> > > below:
> > > >> > > > >
> > > >> > > > > 1) The motivation currently explicitly states that the goal
> is
> > > to
> > > >> > > improve
> > > >> > > > > performance for heavy state application. It seems that the
> > > >> motivation
> > > >> > > can
> > > >> > > > > be stronger with the following use-case. Currently for
> > > MirrorMaker
> > > >> > > > cluster
> > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> time
> > to
> > > >> > > rolling
> > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > process
> > > >> > restart
> > > >> > > > > will trigger a rebalance which currently pause the
> consumption
> > > of
> > > >> the
> > > >> > > all
> > > >> > > > > partitions of the MirrorMaker cluster. With the change
> stated
> > in
> > > >> this
> > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > specified
> > > >> > > timeout
> > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > rebalance
> > > >> > (e.g.
> > > >> > > > for
> > > >> > > > > leader restart) for the entire rolling bounce, which will
> > > >> > significantly
> > > >> > > > > improves the availability of the MirrorMaker pipeline. In my
> > > >> opinion,
> > > >> > > the
> > > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance if
> > the
> > > >> > > consumer
> > > >> > > > > process can be restarted within soon, which helps
> performance
> > > >> even if
> > > >> > > > > overhead of state shuffling for a given process is small.
> > > >> > > > >
> > > >> > > > > 2) In order to simplify the KIP reading, can you follow the
> > > >> writeup
> > > >> > > style
> > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> such
> > as
> > > >> new
> > > >> > > > > configs (e.g. registration timeout), new request/response,
> new
> > > >> > > > AdminClient
> > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > Currently
> > > >> some
> > > >> > > of
> > > >> > > > > these are specified in the Proposed Change section which
> makes
> > > it
> > > >> a
> > > >> > bit
> > > >> > > > > inconvenient to understand the new interface that will be
> > > exposed
> > > >> to
> > > >> > > > user.
> > > >> > > > > Explanation of the current two-phase rebalance protocol
> > probably
> > > >> can
> > > >> > be
> > > >> > > > > moved out of public interface section.
> > > >> > > > >
> > > >> > > > > 3) There are currently two version of JoinGroupRequest in
> the
> > > KIP
> > > >> and
> > > >> > > > only
> > > >> > > > > one of them has field memberId. This seems confusing.
> > > >> > > > >
> > > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > > >> rebalance
> > > >> > > could
> > > >> > > > > be helpful here, but we will make a call once we finished
> the
> > > >> major
> > > >> > > > > implementation". So this seems to be still an open question
> in
> > > the
> > > >> > > > current
> > > >> > > > > design. We probably want to agree on this before voting for
> > the
> > > >> KIP.
> > > >> > > > >
> > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> consumer.
> > > Can
> > > >> > you
> > > >> > > > > specify the name of the config key and the default config
> > value?
> > > >> > > Possible
> > > >> > > > > default values include empty string or null (similar to
> > > >> > transaction.id
> > > >> > > > in
> > > >> > > > > producer config).
> > > >> > > > >
> > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > persist
> > > >> > member
> > > >> > > > > name map, currently if consumer coordinator broker goes
> > offline,
> > > >> > > > rebalance
> > > >> > > > > is triggered and consumers will try connect to the new
> > > >> coordinator.
> > > >> > If
> > > >> > > > > these consumers can connect to the new coordinator within
> > > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> > that
> > > >> > broker
> > > >> > > > can
> > > >> > > > > use a deterministic algorithm to determine the partition ->
> > > >> > member_name
> > > >> > > > > mapping, each consumer should get assigned the same set of
> > > >> partitions
> > > >> > > > > without requiring state shuffling. So it is not clear
> whether
> > we
> > > >> > have a
> > > >> > > > > strong use-case for this new logic. Can you help clarify
> what
> > is
> > > >> the
> > > >> > > > > benefit of using topic "static_member_map" to persist member
> > > name
> > > >> > map?
> > > >> > > > >
> > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > config,
> > > >> it is
> > > >> > > > > mentioned that "we are using expansion timeout to replace
> > > >> rebalance
> > > >> > > > > timeout, which is configured by max.poll.intervals from
> client
> > > >> side,
> > > >> > > and
> > > >> > > > > using registration timeout to replace session timeout".
> > > Currently
> > > >> the
> > > >> > > > > default max.poll.interval.ms is configured to be 5 minutes
> > and
> > > >> there
> > > >> > > > will
> > > >> > > > > be only one rebalance if all new consumers can join within 5
> > > >> minutes.
> > > >> > > So
> > > >> > > > it
> > > >> > > > > is not clear whether we have a strong use-case for this new
> > > >> config.
> > > >> > Can
> > > >> > > > you
> > > >> > > > > explain what is the benefit of introducing this new config?
> > > >> > > > >
> > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > version
> > > >> of
> > > >> > > > > protocol, we will also increase the join group request
> version
> > > to
> > > >> v4
> > > >> > > when
> > > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> > latest
> > > >> (<
> > > >> > > v4),
> > > >> > > > > the join group request shall be downgraded to v3 without
> > setting
> > > >> the
> > > >> > > > member
> > > >> > > > > Id". It is probably simpler to just say that this feature is
> > > >> enabled
> > > >> > if
> > > >> > > > > JoinGroupRequest V4 is supported on both client and broker
> and
> > > >> > > > MEMBER_NAME
> > > >> > > > > is configured with non-empty string.
> > > >> > > > >
> > > >> > > > > 9) It is mentioned that broker may return
> > > >> NO_STATIC_MEMBER_INFO_SET
> > > >> > > error
> > > >> > > > > in OffsetCommitResponse for "commit requests under static
> > > >> > membership".
> > > >> > > > Can
> > > >> > > > > you clarify how broker determines whether the commit request
> > is
> > > >> under
> > > >> > > > > static membership?
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > > Dong
> > > >> > > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > --
> > > >> > > > -- Guozhang
> > > >> > > >
> > > >> > >
> > > >> >
> > > >> >
> > > >> > --
> > > >> > -Regards,
> > > >> > Mayuresh R. Gharat
> > > >> > (862) 250-7125
> > > >> >
> > > >>
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


--
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Guozhang Wang <wa...@gmail.com>.
Regarding Jason's question and Boyang's responses:

2) I once have a discussion about the LeaveGroupRequest for static members,
and the reason for not having it for static members is that we'd need to
make it a configurable behavior as well (i.e. the likelihood that a static
member may shutdown but come back later may be even larger than the
likelihood that a shutdown static member would not come back), and when a
shutdown is complete the instance cannot tell whether or not it will come
back by itself. And hence letting a third party (think: admin used by K8s
plugins) issuing a request to indicate static member changes would be more
plausible.

I think having an optional list of all the static members that are still in
the group, rather than the members to be removed since the latter looks a
bit less flexible to me, in the request is a good idea (remember we allow a
group to have both static and dynamic members at the same time, so when
receiving the request, we will only do the diff and add / remove the static
members directly only, while still let the dynamic members to try to
re-join the group with the rebalance timeout).

3) personally I favor "ids" over "names" :) Since we already have some
"ids" and hence it sounds more consistent, plus on the producer side we
have a `transactional.id` whose semantics is a bit similar to this one,
i.e. for unique distinguishment of a client which may comes and goes but
need to be persist over multiple "instance life-times".


Guozhang


On Tue, Nov 27, 2018 at 10:00 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> Hi Boyang,
>
> Thanks for the replies. Please find the follow up queries below.
>
>     5. Regarding "So in summary, *the member will only be removed due to
> session timeout*. We shall remove it from both in-memory static member name
> mapping and member list." If the rebalance is invoked manually using the
> the admin apis, how long should the group coordinator wait for the members
> of the group to send a JoinGroupRequest for participating in the rebalance?
> How is a lagging consumer handled?
> The plan is to disable member kick out when rebalance.timeout is reached,
> so basically we are not "waiting" any
> join group request from existing members; we shall just rebalance base on
> what we currently have within the group
> metadata. Lagging consumer will trigger rebalance later if session timeout
> > rebalance timeout.
>
> >
> Just wanted to understand this better. Lets take an example, say we have a
> > consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> > Everything is running fine and suddenly C4 host has issues and it goes
> > down. Now we notice that we can still operate with c1, c2, c3 and don't
> > want to wait for
> > c4 to come back up. We use the admin api
> > "invokeConsumerRebalance("GroupA")".
> > Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> > group again (in there heartBeatResponse) as first step of rebalance.
> > Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> > delayed. At this stage, if we are not "waiting" on any join group
> request,
> > few things can happen :
> >
> >    - c4's partitions are distributed only among c1,c2. c3 maintains its
> >    original assignment. c1, c2 will start processing the newly assigned
> >    partitions.
> >
> > OR
> >
> >    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
> >    processing the newly assigned partitions. c3 gets to know about the
> newly
> >    assigned partitions later when it sends the JoinGroupRequest (which
> was
> >    delayed).
> >
> > OR
> >
> >    - Will the rebalance do a complete reassignment, where c1, c2, c3 have
> >    to give up there partitions and all the partitions belonging to c1,
> c2, c3,
> >    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
> >    GroupCoordinator needs to give some buffer time for c1, c2, c3 to
> revoke
> >    there partitions and rejoin the group.
> >
> > This is as per my understanding of how the KIP would work without
> changing
> > the underlying group coordination workflow. Please correct me if I
> > misunderstood something here.
> >
>
>
> - When we say that we would use invokeConsumerRebalance(groupId) to down
> scale, with the example in the above question, how will the
> GroupCoordinator know that c4 should be kicked out of the group since we
> are trying to invoke rebalance proactively without waiting for c4's session
> time out to expire. Should there be a way of telling the GroupCoordinator
> that consumer c4 has been kicked out of the groupId = "GroupA"?
>
> - Also it looks like the statement "If the `member.id` uses
> UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
> the one within current map, if `group.member.name` is known. Also once we
> are done with KIP-394
> <
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member.id+for+initial+join+group+request
> >,
> all the join group requests are requiring `member.id` to physically enter
> the consumer group. This way the latest joined " is incomplete. Can you
> take a look at this?
> Also when we say "all the join group requests are requiring `member.id` to
> physically enter the consumer group." because a newly started consumer will
> not have a "member.id", I assume you mean, once the GroupCoordinator
> assigns a member.id to the newly started consumer, it has to use it for
> any
> future JoinGroupRequests. Is my understanding correct?
>
>
> Thanks,
>
> Mayuresh
>
> On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Mayuresh and Jason for your follow-ups! Let me try to answer both
> > in this reply.
> >
> >
> > >    1. Do you intend to have member.id is a static config like
> > member.name
> > >    after KIP-345 and KIP-394?
> >
> > No, we shall only rely on broker to allocate member.id for the consumer
> > instances. FYI, I already
> >
> > started the discussion thread for KIP-394 😊
> >
> > >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> > in
> > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config
> is
> > > set,
> > >    we will put it in the initial join group request to identify itself
> > as a
> > >    static member (static membership); otherwise, we will still send
> > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > (dynamic
> > >    membership)."
> > >       - What is the value of member_id sent in the first
> JoinGroupRequest
> > >       when member_name is set (using static rebalance)? Is it
> > > UNKNOW_MEMBER_ID?
> >
> > Yes, we could only use unknown member id. Actually this part of the
> > proposal is outdated,
> >
> > let me do another audit of the whole doc. Basically, it is currently
> > impossible to send `member.id`
> >
> > when consumer restarted. Sorry for the confusions!
> >
> > >    3. Regarding "we are requiring member.id (if not unknown) to match
> > the
> > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> > case
> > >    that if we could have members with the same `member.name` (for
> > example
> > >    mis-configured instances with a valid member.id but added a used
> > member
> > >    name on runtime). When member name has duplicates, we could refuse
> > join
> > >    request from members with an outdated `member.id` (since we update
> > the
> > >    mapping upon each join group request). In an edge case where the
> > client
> > >    hits this exception in the response, it is suggesting that some
> other
> > >    consumer takes its spot."
> > >       - The part of "some other consumer takes the spot" would be
> > >       intentional, right? Also when you say " The edge case that if we
> > >       could have members with the same `member.name` (for example
> > >       mis-configured instances *with a valid member.id <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> > >
> > > *but
> > >       added a used member name on runtime).", what do you mean by
> *valid
> > >       member id* here? Does it mean that there exist a mapping of
> > >       member.name to member.id like *MemberA -> id1* on the
> > >       GroupCoordinator and this consumer is trying to join with *
> > > member.name
> > >       <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0
> >
> > = MemberB and member.id <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> >
> > =
> > > id1 *
> > >       ?
> >
> > I would take Jason's advice that each time we have unknown member joining
> > the group, the broker will
> >
> > always assign a new and unique id to track its identity. In this way,
> > consumer with duplicate member name
> >
> > will be fenced.
> >
> > >    4. Depending on your explanation for point 2 and the point 3 above
> > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > >    member_name but unknown member_id, if the consumer sends
> > > "UNKNOW_MEMBER_ID"
> > >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > > give it
> > >    a member_id, is the consumer suppose to remember member_id for
> > >    joinGroupRequests? If yes, how are restarts handled?
> >
> > Like explained above, we shall not materialize the member.id. Instead we
> > need to rely on broker to allocate
> >
> > a unique id for consumer just like what we have now.
> >
> > >    5. Regarding "So in summary, *the member will only be removed due to
> > >    session timeout*. We shall remove it from both in-memory static
> member
> > >    name mapping and member list."
> > >       - If the rebalance is invoked manually using the the admin apis,
> > how
> > >       long should the group coordinator wait for the members of the
> > > group to send
> > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > lagging
> > >       consumer handled?
> >
> > The plan is to disable member kick out when rebalance.timeout is reached,
> > so basically we are not "waiting" any
> >
> > join group request from existing members; we shall just rebalance base on
> > what we currently have within the group
> >
> > metadata. Lagging consumer will trigger rebalance later if session
> timeout
> > > rebalance timeout.
> >
> > >    6. Another detail to take care is that we need to automatically take
> > the
> > >    hash of group id so that we know which broker to send this request
> to.
> > >       - I assume this should be same as the way we find the
> coordinator,
> > >       today right? If yes, should we specify it in the KIP ?
> >
> > Yep, it is. Add FindCoordinatorRequest logic to the script.
> >
> > >    7. Are there any specific failure scenarios when you say "other
> > >    potential failure cases."? It would be good to mention them
> > explicitly,
> > > if
> > >    you think there are any.
> >
> > Nah, I'm gonna remove it because it seems causing more confusion than
> > making my assumption clear, which is
> >
> > "there could be other failure cases that I can't enumerate now" 😊
> >
> > >    8. It would be good to have a rollback plan as you have for roll
> > forward
> > >    in the KIP.
> >
> > Great suggestion! Added a simple rollback plan.
> >
> >
> > Next is answering Jason's suggestions:
> >
> > 1. This may be the same thing that Mayuresh is asking about. I think the
> > suggestion in the KIP is that if a consumer sends JoinGroup with a member
> > name, but no member id, then we will return the current member id
> > associated with that name. It seems in this case that we wouldn't be able
> > to protect from having two consumers active with the same configured
> > member.name? For example, imagine that we had a consumer with
> member.name
> > =A
> > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > instance starts up with member.name=A. If it is also assigned member.id
> =1,
> > then how can we detect the zombie if it comes back to life? Both
> instances
> > will have the same member.id.
> >
> > The goal is to avoid a rebalance on a rolling restart, but we still need
> to
> > fence previous members. I am wondering if we can generate a new
> member.id
> > every time we receive a request from a static member with an unknown
> member
> > id. If the old instance with the same member.name attempts any
> operation,
> > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > subscription of the new instance hasn't changed, then we can skip the
> > rebalance and return the current assignment without forcing a rebalance.
> >
> > The trick to making this work is in the error handling of the zombie
> > consumer. If the zombie simply resets its member.id and rejoins to get a
> > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> > fencing the new member. We want to avoid this. There needs to be an
> > expectation for static members that the member.id of a static member
> will
> > not be changed except when a new member with the same member.name joins
> > the
> > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> > with static member names.
> >
> > Yep, I like this idea! Keep giving out refresh member.id when facing
> > anonymous request will definitely
> >
> > prevent processing bug due to duplicate consumers, however I don't think
> I
> > fully understand the 3rd paragraph where
> >
> > you mentioned  "There needs to be an expectation for static members that
> > the member.id of a static member will
> >
> > not be changed except when a new member with the same member.name joins
> > the group. "  How do you plan
> > to know whether this member is new member or old member? I feel even with
> > zombie consumer takes the ownership,
> > it should be detected very quickly (as MISMATCH_ID exception trigger
> > original consumer instance dies)
> > and end user will start to fix it right away. Is there any similar logic
> > we applied in fencing duplicate `transaction.id`?
> >
> > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far
> as
> > I understand it, it is used for scaling down a consumer group and somehow
> > bypasses normal session timeout expiration. I am wondering how critical
> > this piece is and whether we can leave it for future work. If not, then
> it
> > would be helpful to elaborate on its implementation. How would the
> > coordinator know which members to kick out of the group?
> >
> > This API is needed when we need to immediately trigger rebalance instead
> > of waiting session timeout
> >
> > or rebalance timeout (Emergent scale up/down). It is very necessary to
> > have it for
> >
> > management purpose because user could choose when to trigger rebalance
> > pretty freely,
> >
> > gaining more client side control.
> >
> > In the meanwhile I see your point that we need to actually have the
> > ability to kick out members that we plan
> >
> > to scale down fast (as rebalance timeout no longer kicks any offline
> > member out of the group), I will think of adding an optional
> >
> > list of members that are ready to be removed.
> >
> > Another idea is to let static member send `LeaveGroupRequest` when they
> > are going offline (either scale down or bouncing),
> >
> > and broker will cache this information as "OfflineMembers" without
> > triggering rebalance. When handling ConsumerRebalanceRequest broker will
> >
> > kick the static members that are currently offline and trigger rebalance
> > immediately. How does this plan sound?
> >
> > 3. I've been holding back on mentioning this, but I think we should
> > reconsider the name `member.name`. I think we want something that
> suggests
> > its expectation of uniqueness in the group. How about `group.instance.id
> `
> > to go along with `group.id`?
> >
> > Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> > the namespace idea, and
> >
> > since we already use `member.name` in a lot of context, I decide to
> > rename the config to `group.member.name`
> >
> > which should be sufficient for solving all the concerns we have now.
> > Sounds good?
> >
> >
> > Thank you for your great suggestions! Let me know if my reply makes sense
> > her.
> >
> >
> > Best,
> >
> > Boyang
> >
> > ________________________________
> > From: Jason Gustafson <ja...@confluent.io>
> > Sent: Tuesday, November 27, 2018 7:51 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the updates. Looks like we're headed in the right direction
> and
> > clearly the interest that this KIP is receiving shows how strong the
> > motivation is!
> >
> > I have a few questions:
> >
> > 1. This may be the same thing that Mayuresh is asking about. I think the
> > suggestion in the KIP is that if a consumer sends JoinGroup with a member
> > name, but no member id, then we will return the current member id
> > associated with that name. It seems in this case that we wouldn't be able
> > to protect from having two consumers active with the same configured
> > member.name? For example, imagine that we had a consumer with
> member.name
> > =A
> > which is assigned member.id=1. Suppose it becomes a zombie and a new
> > instance starts up with member.name=A. If it is also assigned member.id
> =1,
> > then how can we detect the zombie if it comes back to life? Both
> instances
> > will have the same member.id.
> >
> > The goal is to avoid a rebalance on a rolling restart, but we still need
> to
> > fence previous members. I am wondering if we can generate a new
> member.id
> > every time we receive a request from a static member with an unknown
> member
> > id. If the old instance with the same member.name attempts any
> operation,
> > then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> > subscription of the new instance hasn't changed, then we can skip the
> > rebalance and return the current assignment without forcing a rebalance.
> >
> > The trick to making this work is in the error handling of the zombie
> > consumer. If the zombie simply resets its member.id and rejoins to get a
> > new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> > fencing the new member. We want to avoid this. There needs to be an
> > expectation for static members that the member.id of a static member
> will
> > not be changed except when a new member with the same member.name joins
> > the
> > group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> > with static member names.
> >
> > 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far
> as
> > I understand it, it is used for scaling down a consumer group and somehow
> > bypasses normal session timeout expiration. I am wondering how critical
> > this piece is and whether we can leave it for future work. If not, then
> it
> > would be helpful to elaborate on its implementation. How would the
> > coordinator know which members to kick out of the group?
> >
> > 3. I've been holding back on mentioning this, but I think we should
> > reconsider the name `member.name`. I think we want something that
> suggests
> > its expectation of uniqueness in the group. How about `group.instance.id
> `
> > to go along with `group.id`?
> >
> > Thanks,
> > Jason
> >
> >
> >
> > On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> > gharatmayuresh15@gmail.com>
> > wrote:
> >
> > > Hi Boyang,
> > >
> > > Thanks a lot for replying to all the queries and discussions here, so
> > > patiently.
> > > Really appreciate it.
> > >
> > > Had a few questions and suggestions after rereading the current version
> > of
> > > the KIP :
> > >
> > >
> > >    1. Do you intend to have member.id is a static config like
> > member.name
> > >    after KIP-345 and KIP-394?
> > >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> > in
> > >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config
> is
> > > set,
> > >    we will put it in the initial join group request to identify itself
> > as a
> > >    static member (static membership); otherwise, we will still send
> > >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> > (dynamic
> > >    membership)."
> > >       - What is the value of member_id sent in the first
> JoinGroupRequest
> > >       when member_name is set (using static rebalance)? Is it
> > > UNKNOW_MEMBER_ID?
> > >    3. Regarding "we are requiring member.id (if not unknown) to match
> > the
> > >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> > case
> > >    that if we could have members with the same `member.name` (for
> > example
> > >    mis-configured instances with a valid member.id but added a used
> > member
> > >    name on runtime). When member name has duplicates, we could refuse
> > join
> > >    request from members with an outdated `member.id` (since we update
> > the
> > >    mapping upon each join group request). In an edge case where the
> > client
> > >    hits this exception in the response, it is suggesting that some
> other
> > >    consumer takes its spot."
> > >       - The part of "some other consumer takes the spot" would be
> > >       intentional, right? Also when you say " The edge case that if we
> > >       could have members with the same `member.name` (for example
> > >       mis-configured instances *with a valid member.id <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> > >
> > > *but
> > >       added a used member name on runtime).", what do you mean by
> *valid
> > >       member id* here? Does it mean that there exist a mapping of
> > >       member.name to member.id like *MemberA -> id1* on the
> > >       GroupCoordinator and this consumer is trying to join with *
> > > member.name
> > >       <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0
> >
> > = MemberB and member.id <
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> >
> > =
> > > id1 *
> > >       ?
> > >    4. Depending on your explanation for point 2 and the point 3 above
> > >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> > >    member_name but unknown member_id, if the consumer sends
> > > "UNKNOW_MEMBER_ID"
> > >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > > give it
> > >    a member_id, is the consumer suppose to remember member_id for
> > >    joinGroupRequests? If yes, how are restarts handled?
> > >    5. Regarding "So in summary, *the member will only be removed due to
> > >    session timeout*. We shall remove it from both in-memory static
> member
> > >    name mapping and member list."
> > >       - If the rebalance is invoked manually using the the admin apis,
> > how
> > >       long should the group coordinator wait for the members of the
> > > group to send
> > >       a JoinGroupRequest for participating in the rebalance? How is a
> > > lagging
> > >       consumer handled?
> > >    6. Another detail to take care is that we need to automatically take
> > the
> > >    hash of group id so that we know which broker to send this request
> to.
> > >       - I assume this should be same as the way we find the
> coordinator,
> > >       today right? If yes, should we specify it in the KIP ?
> > >    7. Are there any specific failure scenarios when you say "other
> > >    potential failure cases."? It would be good to mention them
> > explicitly,
> > > if
> > >    you think there are any.
> > >    8. It would be good to have a rollback plan as you have for roll
> > forward
> > >    in the KIP.
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > > gharatmayuresh15@gmail.com>
> > > wrote:
> > >
> > > > Hi Boyang,
> > > >
> > > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > >> Hey Dong, thanks for the follow-up here!
> > > >>
> > > >>
> > > >> 1) It is not very clear to the user what is the difference between
> > > >> member.name and client.id as both seems to be used to identify the
> > > >> consumer. I am wondering if it would be more intuitive to name it
> > > >> group.member.name (preferred choice since it matches the current
> > > group.id
> > > >> config name) or rebalance.member.name to explicitly show that the
> id
> > is
> > > >> solely used for rebalance.
> > > >> Great question. I feel `member.name` is enough to explain itself,
> it
> > > >> seems not very
> > > >> helpful to make the config name longer. Comparing `name` with `id`
> > gives
> > > >> user the
> > > >> impression that they have the control over it with customized rule
> > than
> > > >> library decided.
> > > >>
> > > >> 2) In the interface change section it is said that
> > > >> GroupMaxSessionTimeoutMs
> > > >> will be changed to 30 minutes. It seems to suggest that we will
> change
> > > the
> > > >> default value of this config. It does not seem necessary to increase
> > the
> > > >> time of consumer failure detection when user doesn't use static
> > > >> membership.
> > > >> Also, say static membership is enabled, then this default config
> > change
> > > >> will cause a partition to be unavailable for consumption for 30
> > minutes
> > > if
> > > >> there is hard consumer failure, which seems to be worse experience
> > than
> > > >> having unnecessary rebalance (when this timeout is small),
> > particularly
> > > >> for
> > > >> new users of Kafka. Could you explain more why we should make this
> > > change?
> > > >> We are not changing the default session timeout value. We are just
> > > >> changing the
> > > >> cap we are enforcing on the session timeout max value. So this
> change
> > is
> > > >> not affecting
> > > >> what kind of membership end user is using, and loosing the cap is
> > giving
> > > >> end user
> > > >> more flexibility on trade-off between liveness and stability.
> > > >>
> > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > DUPLICATE_STATIC_MEMBER
> > > >> into one error? It seems that these two errors are currently handled
> > by
> > > >> the
> > > >> consumer in the same way. And we don't also don't expect
> > > >> MEMBER_ID_MISMATCH
> > > >> to happen. Thus it is not clear what is the benefit of having two
> > > errors.
> > > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error because
> > with
> > > >> the KIP-394<
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=EDM7PmpOo2HenYhFHX2rxrszpkI7di401WhKh2Vjw5k%3D&amp;reserved=0
> > > >> >
> > > >> we will automatically fence all join requests with
> UNKNOWN_MEMBER_ID.
> > > >>
> > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > contains
> > > >> member name which is already in the consumer group, however the
> member
> > > id
> > > >> was missing". After a consumer is restarted, it will send a
> > > >> JoinGroupRequest with an existing memberName (as the coordinator has
> > not
> > > >> expired this member from the memory) and memberId
> > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> persisted
> > > >> across consumer restart in the consumer side). Does it mean that
> > > >> JoinGroupRequest from a newly restarted consumer will always be
> > rejected
> > > >> until the sessionTimeoutMs has passed?
> > > >> Same answer as question 3). This part of the logic shall be removed
> > from
> > > >> the proposal.
> > > >>
> > > >> 5) It seems that we always add two methods to the interface
> > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> and
> > > the
> > > >> other without option. Could this be specified in the interface
> change
> > > >> section?
> > > >> Sounds good! Added both methods.
> > > >>
> > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > trigger
> > > >> rebalance? If so, we probably want to specify the command line tool
> > > >> interface similar to
> > > >>
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> > > >> .
> > > >> Added the script.
> > > >>
> > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > meaning
> > > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > > >> simpler
> > > >> to allows this API to trigger rebalance regardless of whether
> consumer
> > > is
> > > >> configured with memberName.
> > > >> Sounds good. Right now I feel for both static and dynamic membership
> > it
> > > is
> > > >> more manageable to introduce the consumer rebalance method through
> > admin
> > > >> client API.
> > > >>
> > > >> 8) It is not very clear how the newly added AdminClient API trigger
> > > >> rebalance. For example, does it send request? Can this be explained
> in
> > > the
> > > >> KIP?
> > > >>
> > > >> Sure, I will add more details to the API.
> > > >>
> > > >>
> > > >> Thanks again for the helpful suggestions!
> > > >>
> > > >>
> > > >> Best,
> > > >> Boyang
> > > >>
> > > >> ________________________________
> > > >> From: Dong Lin <li...@gmail.com>
> > > >> Sent: Saturday, November 24, 2018 2:54 PM
> > > >> To: dev
> > > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > >> specifying member id
> > > >>
> > > >> Hey Boyang,
> > > >>
> > > >> Thanks for the update! Here are some followup comments:
> > > >>
> > > >> 1) It is not very clear to the user what is the difference between
> > > >> member.name and client.id as both seems to be used to identify the
> > > >> consumer. I am wondering if it would be more intuitive to name it
> > > >> group.member.name (preferred choice since it matches the current
> > > group.id
> > > >> config name) or rebalance.member.name to explicitly show that the
> id
> > is
> > > >> solely used for rebalance.
> > > >>
> > > >> 2) In the interface change section it is said that
> > > >> GroupMaxSessionTimeoutMs
> > > >> will be changed to 30 minutes. It seems to suggest that we will
> change
> > > the
> > > >> default value of this config. It does not seem necessary to increase
> > the
> > > >> time of consumer failure detection when user doesn't use static
> > > >> membership.
> > > >> Also, say static membership is enabled, then this default config
> > change
> > > >> will cause a partition to be unavailable for consumption for 30
> > minutes
> > > if
> > > >> there is hard consumer failure, which seems to be worse experience
> > than
> > > >> having unnecessary rebalance (when this timeout is small),
> > particularly
> > > >> for
> > > >> new users of Kafka. Could you explain more why we should make this
> > > change?
> > > >>
> > > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> > DUPLICATE_STATIC_MEMBER
> > > >> into one error? It seems that these two errors are currently handled
> > by
> > > >> the
> > > >> consumer in the same way. And we don't also don't expect
> > > >> MEMBER_ID_MISMATCH
> > > >> to happen. Thus it is not clear what is the benefit of having two
> > > errors.
> > > >>
> > > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > > contains
> > > >> member name which is already in the consumer group, however the
> member
> > > id
> > > >> was missing". After a consumer is restarted, it will send a
> > > >> JoinGroupRequest with an existing memberName (as the coordinator has
> > not
> > > >> expired this member from the memory) and memberId
> > > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not
> persisted
> > > >> across consumer restart in the consumer side). Does it mean that
> > > >> JoinGroupRequest from a newly restarted consumer will always be
> > rejected
> > > >> until the sessionTimeoutMs has passed?
> > > >>
> > > >> 5) It seems that we always add two methods to the interface
> > > >> org.apache.kafka.clients.admin.AdminClient.java, one with options
> and
> > > the
> > > >> other without option. Could this be specified in the interface
> change
> > > >> section?
> > > >>
> > > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> > trigger
> > > >> rebalance? If so, we probably want to specify the command line tool
> > > >> interface similar to
> > > >>
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> > > >> .
> > > >>
> > > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > > meaning
> > > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > > >> simpler
> > > >> to allows this API to trigger rebalance regardless of whether
> consumer
> > > is
> > > >> configured with memberName.
> > > >>
> > > >> 8) It is not very clear how the newly added AdminClient API trigger
> > > >> rebalance. For example, does it send request? Can this be explained
> in
> > > the
> > > >> KIP?
> > > >>
> > > >> Thanks,
> > > >> Dong
> > > >>
> > > >>
> > > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > >>
> > > >> > Hey Mayuresh,
> > > >> >
> > > >> >
> > > >> > thanks for your feedbacks! I will try do another checklist here.
> > > >> >
> > > >> >
> > > >> > > By this you mean, even if the application has not called
> > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > sending
> > > >> the
> > > >> > > LeaveGroup request, right?
> > > >> >
> > > >> > Yep it's true, we will prevent client from sending leave group
> > request
> > > >> > when they are set with `member.name`.
> > > >> >
> > > >> >
> > > >> > > When is the member.name removed from this map?
> > > >> > Good question, we will only kick off member due to session timeout
> > > >> within
> > > >> > static membership. Let me update the KIP to clearly assert that.
> > > >> >
> > > >> > > How is this case (missing member id) handled on the client side?
> > > What
> > > >> is
> > > >> > the application that
> > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > >> > I have extended the two exceptions within join group response V4.
> > > >> > Basically I define both corresponding actions to be immediate
> > failing
> > > >> > client application, because so far it is unknown what kind of
> client
> > > >> issue
> > > >> > could trigger them. After the first version, we will keep enhance
> > the
> > > >> error
> > > >> > handling logic!
> > > >> >
> > > >> > > This would mean that it might take more time to detect unowned
> > topic
> > > >> > > partitions and may cause delay for applications that perform
> data
> > > >> > mirroring
> > > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > > make
> > > >> > here
> > > >> > > as listed below separately.
> > > >> > The goal of extending session timeout cap is for users with good
> > > client
> > > >> > side monitoring tools that could auto-heal the dead consumers very
> > > >> fast. So
> > > >> > it is optional (and personal) to extend session timeout to a
> > > reasonable
> > > >> > number with different client scenarios.
> > > >> >
> > > >> > > you meant remove unjoined members of the group, right ?
> > > >> > Yep, there is a typo. Thanks for catching this!
> > > >> >
> > > >> > > What do you mean by " Internally we would optimize this logic by
> > > >> having
> > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > stage,
> > > >> > > without removing non-responsive members immediately." There
> would
> > > not
> > > >> be
> > > >> > a
> > > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > > later,
> > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > >> > No, there won't be. We want to limit the rebalance timeout
> > > functionality
> > > >> > to only use as a timer to
> > > >> > end prepare rebalance stage. This way, late joining static members
> > > will
> > > >> > not trigger further rebalance
> > > >> > as long as they are within session timeout. I added your highlight
> > to
> > > >> the
> > > >> > KIP!
> > > >> >
> > > >> > > The KIP talks about scale up scenario but its not quite clear
> how
> > we
> > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > adding
> > > >> > status
> > > >> > > "learner" ?. Can you shed more light on how this is handled in
> the
> > > >> KIP,
> > > >> > if
> > > >> > > its handled?
> > > >> > Updated the KIP: we shall not cover scale up case in 345, because
> we
> > > >> > believe client side could
> > > >> > better handle this logic.
> > > >> >
> > > >> > > I think Jason had brought this up earlier about having a way to
> > say
> > > >> how
> > > >> > > many members/consumer hosts are you choosing to be in the
> consumer
> > > >> group.
> > > >> > > If we can do this, then in case of mirroring applications we can
> > do
> > > >> this
> > > >> > :
> > > >> > > Lets say we have a mirroring application that consumes from
> Kafka
> > > >> cluster
> > > >> > > A and produces to Kafka cluster B.
> > > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > > >> service
> > > >> > > providers can set a mirroring group saying that it will take,
> for
> > > >> example
> > > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > > >> latency
> > > >> > > for mirroring and can have additional 10 consumer hosts as spare
> > in
> > > >> the
> > > >> > > same group.
> > > >> > > So when the first 300 members/consumers to join the group will
> > start
> > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > >> > > The remaining 10 consumer members can sit idle.
> > > >> > > The moment one of the consumer (for example: consumer number 54)
> > > from
> > > >> the
> > > >> > > first 300 members go out of the group (crossed session timeout),
> > it
> > > >> (the
> > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > >> consumer
> > > >> > > member 54 to one of the spare hosts.
> > > >> > > Once the consumer member 54 comes back up, it can start as
> being a
> > > >> part
> > > >> > of
> > > >> > > the spare pool.
> > > >> > > This enables us to have lower session timeouts and low latency
> > > >> mirroring,
> > > >> > > in cases where the service providers are OK with having spare
> > hosts.
> > > >> > > This would mean that we would tolerate n consumer members
> leaving
> > > and
> > > >> > > rejoining the group and still provide low latency as long as n
> <=
> > > >> number
> > > >> > of
> > > >> > > spare consumers.
> > > >> > > If there are no spare host available, we can get back to the
> idea
> > as
> > > >> > > described in the KIP.
> > > >> > Great idea! In fact on top of static membership we could later
> > > introduce
> > > >> > APIs to set hard-coded
> > > >> > client ids to the group and replace the dead host, or as you
> > proposed
> > > to
> > > >> > define spare host as
> > > >> > what I understood as hot backup. I will put both Jason and your
> > > >> > suggestions into a separate section
> > > >> > called "Future works". Note that this spare host idea may be also
> > > >> solvable
> > > >> > through rebalance protocol
> > > >> > IMO.
> > > >> >
> > > >> > Thank you again for the great feedback!
> > > >> >
> > > >> > Boyang
> > > >> > ________________________________
> > > >> > From: Boyang Chen <bc...@outlook.com>
> > > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > >> > specifying member id
> > > >> >
> > > >> > Hey Dong, sorry for missing your message. I couldn't find your
> email
> > > on
> > > >> my
> > > >> > thread, so I will just do a checklist here!
> > > >> >
> > > >> >
> > > >> > 1) The motivation currently explicitly states that the goal is to
> > > >> improve
> > > >> >
> > > >> > performance for heavy state application. It seems that the
> > motivation
> > > >> can
> > > >> >
> > > >> > be stronger with the following use-case. Currently for MirrorMaker
> > > >> cluster
> > > >> >
> > > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > rolling
> > > >> >
> > > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > > restart
> > > >> >
> > > >> > will trigger a rebalance which currently pause the consumption of
> > the
> > > >> all
> > > >> >
> > > >> > partitions of the MirrorMaker cluster. With the change stated in
> > this
> > > >> >
> > > >> > patch, as long as a MirrorMaker can restart within the specified
> > > timeout
> > > >> >
> > > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> > (e.g.
> > > >> for
> > > >> >
> > > >> > leader restart) for the entire rolling bounce, which will
> > > significantly
> > > >> >
> > > >> > improves the availability of the MirrorMaker pipeline. In my
> > opinion,
> > > >> the
> > > >> >
> > > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > >> consumer
> > > >> >
> > > >> > process can be restarted within soon, which helps performance even
> > if
> > > >> >
> > > >> > overhead of state shuffling for a given process is small.
> > > >> >
> > > >> > I just rephrased this part and added it to the KIP. Thanks for
> > making
> > > >> the
> > > >> > motivation more solid!
> > > >> >
> > > >> > 2) In order to simplify the KIP reading, can you follow the
> writeup
> > > >> style
> > > >> > of other KIP (e.g. KIP-98) and list the interface change such as
> new
> > > >> > configs (e.g. registration timeout), new request/response, new
> > > >> AdminClient
> > > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> > some
> > > of
> > > >> > these are specified in the Proposed Change section which makes it
> a
> > > bit
> > > >> > inconvenient to understand the new interface that will be exposed
> to
> > > >> user.
> > > >> > Explanation of the current two-phase rebalance protocol probably
> can
> > > be
> > > >> > moved out of public interface section.
> > > >> > This is a great suggestion! I just consolidated all the public API
> > > >> > changes, and the whole KIP
> > > >> > looks much more organized!
> > > >> >
> > > >> > 3) There are currently two version of JoinGroupRequest in the KIP
> > and
> > > >> only
> > > >> > one of them has field memberId. This seems confusing.
> > > >> > Yep, I already found this issue and fixed it.
> > > >> >
> > > >> > 4) It is mentioned in the KIP that "An admin API to force
> rebalance
> > > >> could
> > > >> > be helpful here, but we will make a call once we finished the
> major
> > > >> > implementation". So this seems to be still an open question in the
> > > >> current
> > > >> > design. We probably want to agree on this before voting for the
> KIP.
> > > >> > We have finalized the idea that this API is needed.
> > > >> >
> > > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> > you
> > > >> > specify the name of the config key and the default config value?
> > > >> Possible
> > > >> > default values include empty string or null (similar to
> > > transaction.id<
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=N8RxV6%2Bh7ib9CMpW3ZyFq3m2awY1sRPHzlOTi6qU5XY%3D&amp;reserved=0
> > > >> >
> > > >> > in
> > > >> > producer config).
> > > >> > I have defined the `member.name` in "New configuration" section.
> > > >> >
> > > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > > member
> > > >> > name map, currently if consumer coordinator broker goes offline,
> > > >> rebalance
> > > >> > is triggered and consumers will try connect to the new
> coordinator.
> > If
> > > >> > these consumers can connect to the new coordinator within
> > > >> > max.poll.interval.ms<
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> > > >> >
> > > >> > which by default is 5 minutes, given that broker can
> > > >> > use a deterministic algorithm to determine the partition ->
> > > member_name
> > > >> > mapping, each consumer should get assigned the same set of
> > partitions
> > > >> > without requiring state shuffling. So it is not clear whether we
> > have
> > > a
> > > >> > strong use-case for this new logic. Can you help clarify what is
> the
> > > >> > benefit of using topic "static_member_map" to persist member name
> > map?
> > > >> > I have discussed with Guozhang offline, and I believe reusing the
> > > >> current
> > > >> > `_consumer_offsets`
> > > >> > topic is a better and unified solution.
> > > >> >
> > > >> > 7) Regarding the introduction of the expensionTimeoutMs config, it
> > is
> > > >> > mentioned that "we are using expansion timeout to replace
> rebalance
> > > >> > timeout, which is configured by max.poll.intervals from client
> side,
> > > and
> > > >> > using registration timeout to replace session timeout". Currently
> > the
> > > >> > default max.poll.interval.ms<
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> > > >> >
> > > >> > is configured to be 5 minutes and there will
> > > >> > be only one rebalance if all new consumers can join within 5
> > minutes.
> > > >> So it
> > > >> > is not clear whether we have a strong use-case for this new
> config.
> > > Can
> > > >> you
> > > >> > explain what is the benefit of introducing this new config?
> > > >> > Previously our goal is to use expansion timeout as a workaround
> for
> > > >> > triggering multiple
> > > >> > rebalances when scaling up members are not joining at the same
> time.
> > > It
> > > >> is
> > > >> > decided to
> > > >> > be addressed by client side protocol change, so we will not
> > introduce
> > > >> > expansion timeout.
> > > >> >
> > > >> > 8) It is mentioned that "To distinguish between previous version
> of
> > > >> > protocol, we will also increase the join group request version to
> v4
> > > >> when
> > > >> > MEMBER_NAME is set" and "If the broker version is not the latest
> (<
> > > v4),
> > > >> > the join group request shall be downgraded to v3 without setting
> the
> > > >> member
> > > >> > Id". It is probably simpler to just say that this feature is
> enabled
> > > if
> > > >> > JoinGroupRequest V4 is supported on both client and broker and
> > > >> MEMBER_NAME
> > > >> > is configured with non-empty string.
> > > >> > Yep, addressed this!
> > > >> >
> > > >> > 9) It is mentioned that broker may return
> NO_STATIC_MEMBER_INFO_SET
> > > >> error
> > > >> > in OffsetCommitResponse for "commit requests under static
> > membership".
> > > >> Can
> > > >> > you clarify how broker determines whether the commit request is
> > under
> > > >> > static membership?
> > > >> >
> > > >> > We have agreed that commit request shouldn't be affected by the
> new
> > > >> > membership, thus
> > > >> > removing it here. Thanks for catching this!
> > > >> >
> > > >> > Let me know if you have further suggestions or concerns. Thank you
> > for
> > > >> > your valuable feedback
> > > >> > to help me design the KIP better! (And I will try to address your
> > > >> > feedbacks in next round Mayuresh ??)
> > > >> >
> > > >> > Best,
> > > >> > Boyang
> > > >> > ________________________________
> > > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > by
> > > >> > specifying member id
> > > >> >
> > > >> > Hi Boyang,
> > > >> >
> > > >> > Thanks for updating the KIP. This is a step good direction for
> > > stateful
> > > >> > applications and also mirroring applications whose latency is
> > affected
> > > >> due
> > > >> > to the rebalance issues that we have today.
> > > >> >
> > > >> > I had a few questions on the current version of the KIP :
> > > >> > For the effectiveness of the KIP, consumer with member.name set
> > will
> > > >> *not
> > > >> > send leave group request* when they go offline
> > > >> >
> > > >> > > By this you mean, even if the application has not called
> > > >> > > KafkaConsumer.poll() within session timeout, it will not be
> > sending
> > > >> the
> > > >> > > LeaveGroup request, right?
> > > >> > >
> > > >> >
> > > >> > Broker will maintain an in-memory mapping of {member.name ?
> > member.id
> > > }
> > > >> to
> > > >> > track member uniqueness.
> > > >> >
> > > >> > > When is the member.name removed from this map?
> > > >> > >
> > > >> >
> > > >> > Member.id must be set if the *member.name <
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0
> > > >> >
> > > >> > *is already
> > > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > > >> >
> > > >> > > How is this case handled on the client side? What is the
> > application
> > > >> that
> > > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > > >> > >
> > > >> >
> > > >> > Session timeout is the timeout we will trigger rebalance when a
> > member
> > > >> goes
> > > >> > offline for too long (not sending heartbeat request). To make
> static
> > > >> > membership effective, we should increase the default max session
> > > >> timeout to
> > > >> > 30 min so that end user could config it freely.
> > > >> >
> > > >> > > This would mean that it might take more time to detect unowned
> > topic
> > > >> > > partitions and may cause delay for applications that perform
> data
> > > >> > mirroring
> > > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > > make
> > > >> > here
> > > >> > > as listed below separately.
> > > >> > >
> > > >> >
> > > >> > Currently there is a config called *rebalance timeout* which is
> > > >> configured
> > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > interval
> > > >> is
> > > >> > because consumer could only send request within the call of poll()
> > and
> > > >> we
> > > >> > want to wait sufficient time for the join group request. When
> > reaching
> > > >> > rebalance timeout, the group will move towards completingRebalance
> > > stage
> > > >> > and remove unjoined groups
> > > >> >
> > > >> > > you meant remove unjoined members of the group, right ?
> > > >> > >
> > > >> >
> > > >> > Currently there is a config called *rebalance timeout* which is
> > > >> configured
> > > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > > interval
> > > >> is
> > > >> > because consumer could only send request within the call of poll()
> > and
> > > >> we
> > > >> > want to wait sufficient time for the join group request. When
> > reaching
> > > >> > rebalance timeout, the group will move towards completingRebalance
> > > stage
> > > >> > and remove unjoined groups. This is actually conflicting with the
> > > >> design of
> > > >> > static membership, because those temporarily unavailable members
> > will
> > > >> > potentially reattempt the join group and trigger extra rebalances.
> > > >> > Internally we would optimize this logic by having rebalance
> timeout
> > > >> only in
> > > >> > charge of stopping prepare rebalance stage, without removing
> > > >> non-responsive
> > > >> > members immediately.
> > > >> >
> > > >> > > What do you mean by " Internally we would optimize this logic by
> > > >> having
> > > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > > stage,
> > > >> > > without removing non-responsive members immediately." There
> would
> > > not
> > > >> be
> > > >> > a
> > > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > > later,
> > > >> > > right ? If yes, can you highlight this in the KIP ?
> > > >> > >
> > > >> >
> > > >> > Scale Up
> > > >> >
> > > >> > > The KIP talks about scale up scenario but its not quite clear
> how
> > we
> > > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> > adding
> > > >> > status
> > > >> > > "learner" ?. Can you shed more light on how this is handled in
> the
> > > >> KIP,
> > > >> > if
> > > >> > > its handled?
> > > >> > >
> > > >> >
> > > >> >
> > > >> > *Discussion*
> > > >> > Larger session timeouts causing latency rise for getting data for
> > > >> un-owned
> > > >> > topic partitions :
> > > >> >
> > > >> > > I think Jason had brought this up earlier about having a way to
> > say
> > > >> how
> > > >> > > many members/consumer hosts are you choosing to be in the
> consumer
> > > >> group.
> > > >> > > If we can do this, then in case of mirroring applications we can
> > do
> > > >> this
> > > >> > :
> > > >> > > Lets say we have a mirroring application that consumes from
> Kafka
> > > >> cluster
> > > >> > > A and produces to Kafka cluster B.
> > > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > > >> service
> > > >> > > providers can set a mirroring group saying that it will take,
> for
> > > >> example
> > > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > > >> latency
> > > >> > > for mirroring and can have additional 10 consumer hosts as spare
> > in
> > > >> the
> > > >> > > same group.
> > > >> > > So when the first 300 members/consumers to join the group will
> > start
> > > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > >> > > The remaining 10 consumer members can sit idle.
> > > >> > > The moment one of the consumer (for example: consumer number 54)
> > > from
> > > >> the
> > > >> > > first 300 members go out of the group (crossed session timeout),
> > it
> > > >> (the
> > > >> > > groupCoordinator) can just assign the topicPartitions from the
> > > >> consumer
> > > >> > > member 54 to one of the spare hosts.
> > > >> > > Once the consumer member 54 comes back up, it can start as
> being a
> > > >> part
> > > >> > of
> > > >> > > the spare pool.
> > > >> > > This enables us to have lower session timeouts and low latency
> > > >> mirroring,
> > > >> > > in cases where the service providers are OK with having spare
> > hosts.
> > > >> > > This would mean that we would tolerate n consumer members
> leaving
> > > and
> > > >> > > rejoining the group and still provide low latency as long as n
> <=
> > > >> number
> > > >> > of
> > > >> > > spare consumers.
> > > >> > > If there are no spare host available, we can get back to the
> idea
> > as
> > > >> > > described in the KIP.
> > > >> > >
> > > >> >
> > > >> > Thanks,
> > > >> >
> > > >> > Mayuresh
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> >
> > > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > > >> > konstantine@confluent.io> wrote:
> > > >> >
> > > >> > > Hi Boyang.
> > > >> > >
> > > >> > > Thanks for preparing this KIP! It is making good progress and
> will
> > > be
> > > >> a
> > > >> > > great improvement for stateful Kafka applications.
> > > >> > >
> > > >> > > Apologies for my late reply, I was away for a while. Lots of
> great
> > > >> > comments
> > > >> > > so far, so I'll probably second most of them in what I suggest
> > below
> > > >> at
> > > >> > > this point.
> > > >> > >
> > > >> > > When I first read the KIP, I wanted to start at the end with
> > > something
> > > >> > that
> > > >> > > wasn't highlighted a lot. That was the topic related to handling
> > > >> > duplicate
> > > >> > > members. I see now that the initial suggestion of handling this
> > > >> situation
> > > >> > > during offset commit has been removed, and I agree with that.
> > Issues
> > > >> > > related to membership seem to be handled better when the member
> > > joins
> > > >> the
> > > >> > > group rather than when it tries to commit offsets. This also
> > > >> simplifies
> > > >> > how
> > > >> > > many request types need to change in order to incorporate the
> new
> > > >> member
> > > >> > > name field.
> > > >> > >
> > > >> > > I also agree with what Jason and Guozhang have said regarding
> > > >> timeouts.
> > > >> > > Although semantically, it's easier to think of every operation
> > > having
> > > >> its
> > > >> > > own timeout, operationally this can become a burden. Thus,
> > > >> consolidation
> > > >> > > seems preferable here. The definition of embedded protocols on
> top
> > > of
> > > >> the
> > > >> > > base group membership protocol for rebalancing gives enough
> > > >> flexibility
> > > >> > to
> > > >> > > address such needs in each client component separately.
> > > >> > >
> > > >> > > Finally, some minor comments:
> > > >> > > In a few places the new/proposed changes are referred to as
> > > "current".
> > > >> > > Which is a bit confusing considering that there is a protocol in
> > > place
> > > >> > > already, and by "current" someone might understand the existing
> > one.
> > > >> I'd
> > > >> > > recommend using new/proposed or equivalent when referring to
> > changes
> > > >> > > introduced with KIP-345 and current/existing or equivalent when
> > > >> referring
> > > >> > > to existing behavior.
> > > >> > >
> > > >> > > There's the following sentence in the "Public Interfaces"
> section:
> > > >> > > "Since for many stateful consumer/stream applications, the state
> > > >> > shuffling
> > > >> > > is more painful than short time partial unavailability."
> > > >> > > However, my understanding is that the changes proposed with
> > KIP-345
> > > >> will
> > > >> > > not exploit any partial availability. A suggestion for dealing
> > with
> > > >> > > temporary imbalances has been made in "Incremental Cooperative
> > > >> > Rebalancing"
> > > >> > > which can work well with KIP-345, but here I don't see proposed
> > > >> changes
> > > >> > > that suggest that some resources (e.g. partitions) will keep
> being
> > > >> used
> > > >> > > while others will not be utilized. Thus, you might want to
> adjust
> > > this
> > > >> > > sentence. Correct me if I'm missing something related to that.
> > > >> > >
> > > >> > > In the rejected alternatives, under point 2) I read "we can copy
> > the
> > > >> > member
> > > >> > > id to the config files". I believe it means to say "member name"
> > > >> unless
> > > >> > I'm
> > > >> > > missing something about reusing member ids. Also below I read:
> "By
> > > >> > allowing
> > > >> > > consumers to optionally specifying a member id" which probably
> > > implies
> > > >> > > "member name" again. In a sense this section highlights a
> > potential
> > > >> > > confusion between member name and member id. I wonder if we
> could
> > > >> come up
> > > >> > > with a better term for the new field. StaticTag, StaticLabel, or
> > > even
> > > >> > > StaticName are some suggestions that could potentially help with
> > > >> > confusion
> > > >> > > between MemberId and MemberName and what corresponds to what.
> But
> > I
> > > >> > > wouldn't like to disrupt the discussion with naming conventions
> > too
> > > >> much
> > > >> > at
> > > >> > > this point. I just mention it here as a thought.
> > > >> > >
> > > >> > > Looking forward to see the final details of this KIP. Great work
> > so
> > > >> far!
> > > >> > >
> > > >> > > Konstantine
> > > >> > >
> > > >> > >
> > > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <
> bchen11@outlook.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Thanks Guozhang for the great summary here, and I have been
> > > >> following
> > > >> > up
> > > >> > > > the action items here.
> > > >> > > >
> > > >> > > >
> > > >> > > >   1.  I already updated the KIP to remove the expansion
> timeout
> > > and
> > > >> > > > registration timeout. Great to see them being addressed in
> > client
> > > >> side!
> > > >> > > >   2.  I double checked the design and I believe that it is ok
> to
> > > >> have
> > > >> > > both
> > > >> > > > static member and dynamic member co-exist in the same group.
> So
> > > the
> > > >> > > upgrade
> > > >> > > > shouldn't be destructive and we are removing the two
> membership
> > > >> > protocol
> > > >> > > > switching APIs.
> > > >> > > >   3.  I only have question about this one. I'm still reading
> the
> > > >> > > KafkaApis
> > > >> > > > code here. Should I just use the same authorization logic for
> > > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > > suggested,
> > > >> > this
> > > >> > > > feature could be better addressed in a separate KIP because it
> > is
> > > >> > pretty
> > > >> > > > independent. I could start drafting the KIP once the current
> > > >> proposal
> > > >> > is
> > > >> > > > approved.
> > > >> > > >   5.  I believe that we don't need fencing in offset commit
> > > request,
> > > >> > > since
> > > >> > > > duplicate member.name issue could be handled by join group
> > > >> request. We
> > > >> > > > shall reject join group with known member name but no member
> id
> > > >> (which
> > > >> > > > means we already have an active member using this identity).
> > > >> > > >   6.  I agree to remove that internal config once we move
> > forward
> > > >> with
> > > >> > > > static membership. And I already removed the entire section
> from
> > > the
> > > >> > KIP.
> > > >> > > >
> > > >> > > > Let me know if you have other concerns.
> > > >> > > >
> > > >> > > > Best,
> > > >> > > > Boyang
> > > >> > > > ________________________________
> > > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > >> > > > To: dev
> > > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > >> by
> > > >> > > > specifying member id
> > > >> > > >
> > > >> > > > Hello Boyang,
> > > >> > > >
> > > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> > appreciate
> > > >> your
> > > >> > > > patience answering to the feedbacks from the community. I'd
> like
> > > to
> > > >> add
> > > >> > > my
> > > >> > > > 2cents here:
> > > >> > > >
> > > >> > > > 1. By introducing another two timeout configs,
> > > registration_timeout
> > > >> and
> > > >> > > > expansion_timeout, we are effectively having four timeout
> > configs:
> > > >> > > session
> > > >> > > > timeout, rebalance timeout (configured as "
> max.poll.interval.ms
> > "
> > > on
> > > >> > > client
> > > >> > > > side), and these two. Interplaying these timeout configs can
> be
> > > >> quite
> > > >> > > hard
> > > >> > > > for users with such complexity, and hence I'm wondering if we
> > can
> > > >> > > simplify
> > > >> > > > the situation with as less possible timeout configs as
> possible.
> > > >> Here
> > > >> > is
> > > >> > > a
> > > >> > > > concrete suggestion I'd like propose:
> > > >> > > >
> > > >> > > > 1.a) Instead of introducing a registration_timeout in addition
> > to
> > > >> the
> > > >> > > > session_timeout for static members, we can just reuse the
> > > >> > session_timeout
> > > >> > > > and ask users to set it to a larger value when they are
> > upgrading
> > > a
> > > >> > > dynamic
> > > >> > > > client to a static client by setting the "member.name" at the
> > > same
> > > >> > time.
> > > >> > > > By
> > > >> > > > default, the broker-side min.session.timeout is 6 seconds and
> > > >> > > > max.session.timeout is 5 minutes, which seems reasonable to me
> > (we
> > > >> can
> > > >> > of
> > > >> > > > course modify this broker config to enlarge the valid interval
> > if
> > > we
> > > >> > want
> > > >> > > > in practice). And then we should also consider removing the
> > > >> condition
> > > >> > for
> > > >> > > > marking a client as failed if the rebalance timeout has
> reached
> > > >> while
> > > >> > the
> > > >> > > > JoinGroup was not received, so that the semantics of
> > > session_timeout
> > > >> > and
> > > >> > > > rebalance_timeout are totally separated: the former is only
> used
> > > to
> > > >> > > > determine if a consumer member of the group should be marked
> as
> > > >> failed
> > > >> > > and
> > > >> > > > kicked out of the group, and the latter is only used to
> > determine
> > > >> the
> > > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> > phase.
> > > In
> > > >> > > other
> > > >> > > > words if a member did not send the JoinGroup in time of the
> > > >> > > > rebalance_timeout, we still include it in the new generation
> of
> > > the
> > > >> > group
> > > >> > > > and use its old subscription info to send to leader for
> > > assignment.
> > > >> > Later
> > > >> > > > if the member came back with HeartBeat request, we can still
> > > follow
> > > >> the
> > > >> > > > normal path to bring it to the latest generation while
> checking
> > > that
> > > >> > its
> > > >> > > > sent JoinGroup request contains the same subscription info as
> we
> > > >> used
> > > >> > to
> > > >> > > > assign the partitions previously (which should be likely the
> > case
> > > in
> > > >> > > > practice). In addition, we should let static members to not
> send
> > > the
> > > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > > static
> > > >> > > member
> > > >> > > > can only be leaving the group if its session has timed out, OR
> > it
> > > >> has
> > > >> > > been
> > > >> > > > indicated to not exist in the group any more (details below).
> > > >> > > >
> > > >> > > > 1.b) We have a parallel discussion about Incremental
> Cooperative
> > > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> > logic
> > > >> at
> > > >> > the
> > > >> > > > application level, instead of at the protocol level. By doing
> > this
> > > >> we
> > > >> > can
> > > >> > > > also enable a few other optimizations, e.g. at the Streams
> level
> > > to
> > > >> > first
> > > >> > > > build up the state store as standby tasks and then trigger a
> > > second
> > > >> > > > rebalance to actually migrate the active tasks while keeping
> the
> > > >> actual
> > > >> > > > rebalance latency and hence unavailability window to be small
> (
> > > >> > > >
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=OAmsz8pz4JW%2BayjLqwk04E16G%2FTCF%2BbVk0LNB%2BUJgeY%3D&amp;reserved=0
> > > >> > > ).
> > > >> > > > I'd propose we align
> > > >> > > > KIP-345 along with this idea, and hence do not add the
> > > >> > expansion_timeout
> > > >> > > as
> > > >> > > > part of the protocol layer, but only do that at the
> > application's
> > > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > > still,
> > > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > >> > > > <
> > > >> > > >
> > > >> > >
> > > >> >
> > > >>
> > >
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=S8Ds6UNM56%2B3rq%2F%2BvYwKdMmzXrH1B5D3ghyjm06hu4g%3D&amp;reserved=0
> > > >> > > >*"
> > > >> > > > though as part of this KIP
> > > >> > > > since we have discussed about its limit and think it is
> actually
> > > >> not a
> > > >> > > very
> > > >> > > > good design and could be replaced with client-side logic
> above.
> > > >> > > >
> > > >> > > >
> > > >> > > > 2. I'd like to see your thoughts on the upgrade path for this
> > KIP.
> > > >> More
> > > >> > > > specifically, let's say after we have upgraded broker version
> to
> > > be
> > > >> > able
> > > >> > > to
> > > >> > > > recognize the new versions of JoinGroup request and the admin
> > > >> requests,
> > > >> > > how
> > > >> > > > should we upgrade the clients and enable static groups? On top
> > of
> > > my
> > > >> > head
> > > >> > > > if we do a rolling bounce in which we set the member.name
> > config
> > > as
> > > >> > well
> > > >> > > > as
> > > >> > > > optionally increase the session.timeout config when we bounce
> > each
> > > >> > > > instance, then during this rolling bounces we will have a
> group
> > > >> > contained
> > > >> > > > with both dynamic members and static members. It means that we
> > > >> should
> > > >> > > have
> > > >> > > > the group to allow such scenario (i.e. we cannot reject
> > JoinGroup
> > > >> > > requests
> > > >> > > > from dynamic members), and hence the "member.name" -> "
> > member.id"
> > > >> > > mapping
> > > >> > > > will only be partial at this scenario. Also could you describe
> > if
> > > >> the
> > > >> > > > upgrade to the first version that support this feature would
> > ever
> > > >> get
> > > >> > any
> > > >> > > > benefits, or only the future upgrade path for rolling bounces
> > > could
> > > >> get
> > > >> > > > benefits out of this feature?
> > > >> > > >
> > > >> > > > If that's the case and we will do 1) as suggested above, do we
> > > still
> > > >> > need
> > > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > > >> requests
> > > >> > any
> > > >> > > > more? Seems it is not necessary any more as we will only have
> > the
> > > >> > notion
> > > >> > > of
> > > >> > > > "dynamic or static members" that can co-exist in a group while
> > > >> there no
> > > >> > > > notion of "dynamic or static groups", and hence these two
> > requests
> > > >> are
> > > >> > > not
> > > >> > > > needed anymore.
> > > >> > > >
> > > >> > > >
> > > >> > > > 3. We need to briefly talk about the implications for ACL as
> we
> > > >> > introduce
> > > >> > > > new admin requests that are related to a specific group.id.
> For
> > > >> > example,
> > > >> > > > we
> > > >> > > > need to make sure that whoever created the group or joined the
> > > group
> > > >> > can
> > > >> > > > actually send admin requests for the group, otherwise the
> > > >> application
> > > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > > cluster
> > > >> > every
> > > >> > > > time they want to send any admin requests for their groups
> which
> > > >> would
> > > >> > be
> > > >> > > > an operational nightmare.
> > > >> > > >
> > > >> > > >
> > > >> > > > 4. I like Jason's suggestion of adding an optional field for
> the
> > > >> list
> > > >> > of
> > > >> > > > member names, and I'm wondering if that can be done as part of
> > the
> > > >> > > > forceStaticRebalance request: i.e. by passing a list of
> members,
> > > we
> > > >> > will
> > > >> > > > enforce a rebalance immediately since it indicates that some
> > > static
> > > >> > > member
> > > >> > > > will be officially kicked out of the group and some new static
> > > >> members
> > > >> > > may
> > > >> > > > be added. So back to 1.a) above, a static member can only be
> > > kicked
> > > >> out
> > > >> > > of
> > > >> > > > the group if a) its session (arguably long period of time) has
> > > timed
> > > >> > out,
> > > >> > > > and b) this admin request explicitly state that it is no
> longer
> > > >> part of
> > > >> > > the
> > > >> > > > group. As for execution I'm fine with keeping it as a future
> > work
> > > of
> > > >> > this
> > > >> > > > KIP if you'd like to make its scope smaller.
> > > >> > > >
> > > >> > > > Following are minor comments:
> > > >> > > >
> > > >> > > > 5. I'm not sure if we need to include "member.name" as part
> of
> > > the
> > > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> > memberId
> > > >> plus
> > > >> > > the
> > > >> > > > generation number should be sufficient for fencing even with
> > > static
> > > >> > > > members.
> > > >> > > >
> > > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid of
> > the
> > > "
> > > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > >> > > >
> > > >> > > >
> > > >> > > > Guozhang
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <lindong28@gmail.com
> >
> > > >> wrote:
> > > >> > > >
> > > >> > > > > Hey Boyang,
> > > >> > > > >
> > > >> > > > > Thanks for the proposal! This is very useful. I have some
> > > comments
> > > >> > > below:
> > > >> > > > >
> > > >> > > > > 1) The motivation currently explicitly states that the goal
> is
> > > to
> > > >> > > improve
> > > >> > > > > performance for heavy state application. It seems that the
> > > >> motivation
> > > >> > > can
> > > >> > > > > be stronger with the following use-case. Currently for
> > > MirrorMaker
> > > >> > > > cluster
> > > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long
> time
> > to
> > > >> > > rolling
> > > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> > process
> > > >> > restart
> > > >> > > > > will trigger a rebalance which currently pause the
> consumption
> > > of
> > > >> the
> > > >> > > all
> > > >> > > > > partitions of the MirrorMaker cluster. With the change
> stated
> > in
> > > >> this
> > > >> > > > > patch, as long as a MirrorMaker can restart within the
> > specified
> > > >> > > timeout
> > > >> > > > > (e.g. 2 minutes), then we only need constant number of
> > rebalance
> > > >> > (e.g.
> > > >> > > > for
> > > >> > > > > leader restart) for the entire rolling bounce, which will
> > > >> > significantly
> > > >> > > > > improves the availability of the MirrorMaker pipeline. In my
> > > >> opinion,
> > > >> > > the
> > > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance if
> > the
> > > >> > > consumer
> > > >> > > > > process can be restarted within soon, which helps
> performance
> > > >> even if
> > > >> > > > > overhead of state shuffling for a given process is small.
> > > >> > > > >
> > > >> > > > > 2) In order to simplify the KIP reading, can you follow the
> > > >> writeup
> > > >> > > style
> > > >> > > > > of other KIP (e.g. KIP-98) and list the interface change
> such
> > as
> > > >> new
> > > >> > > > > configs (e.g. registration timeout), new request/response,
> new
> > > >> > > > AdminClient
> > > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> > Currently
> > > >> some
> > > >> > > of
> > > >> > > > > these are specified in the Proposed Change section which
> makes
> > > it
> > > >> a
> > > >> > bit
> > > >> > > > > inconvenient to understand the new interface that will be
> > > exposed
> > > >> to
> > > >> > > > user.
> > > >> > > > > Explanation of the current two-phase rebalance protocol
> > probably
> > > >> can
> > > >> > be
> > > >> > > > > moved out of public interface section.
> > > >> > > > >
> > > >> > > > > 3) There are currently two version of JoinGroupRequest in
> the
> > > KIP
> > > >> and
> > > >> > > > only
> > > >> > > > > one of them has field memberId. This seems confusing.
> > > >> > > > >
> > > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > > >> rebalance
> > > >> > > could
> > > >> > > > > be helpful here, but we will make a call once we finished
> the
> > > >> major
> > > >> > > > > implementation". So this seems to be still an open question
> in
> > > the
> > > >> > > > current
> > > >> > > > > design. We probably want to agree on this before voting for
> > the
> > > >> KIP.
> > > >> > > > >
> > > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for
> consumer.
> > > Can
> > > >> > you
> > > >> > > > > specify the name of the config key and the default config
> > value?
> > > >> > > Possible
> > > >> > > > > default values include empty string or null (similar to
> > > >> > transaction.id
> > > >> > > > in
> > > >> > > > > producer config).
> > > >> > > > >
> > > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> > persist
> > > >> > member
> > > >> > > > > name map, currently if consumer coordinator broker goes
> > offline,
> > > >> > > > rebalance
> > > >> > > > > is triggered and consumers will try connect to the new
> > > >> coordinator.
> > > >> > If
> > > >> > > > > these consumers can connect to the new coordinator within
> > > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> > that
> > > >> > broker
> > > >> > > > can
> > > >> > > > > use a deterministic algorithm to determine the partition ->
> > > >> > member_name
> > > >> > > > > mapping, each consumer should get assigned the same set of
> > > >> partitions
> > > >> > > > > without requiring state shuffling. So it is not clear
> whether
> > we
> > > >> > have a
> > > >> > > > > strong use-case for this new logic. Can you help clarify
> what
> > is
> > > >> the
> > > >> > > > > benefit of using topic "static_member_map" to persist member
> > > name
> > > >> > map?
> > > >> > > > >
> > > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> > config,
> > > >> it is
> > > >> > > > > mentioned that "we are using expansion timeout to replace
> > > >> rebalance
> > > >> > > > > timeout, which is configured by max.poll.intervals from
> client
> > > >> side,
> > > >> > > and
> > > >> > > > > using registration timeout to replace session timeout".
> > > Currently
> > > >> the
> > > >> > > > > default max.poll.interval.ms is configured to be 5 minutes
> > and
> > > >> there
> > > >> > > > will
> > > >> > > > > be only one rebalance if all new consumers can join within 5
> > > >> minutes.
> > > >> > > So
> > > >> > > > it
> > > >> > > > > is not clear whether we have a strong use-case for this new
> > > >> config.
> > > >> > Can
> > > >> > > > you
> > > >> > > > > explain what is the benefit of introducing this new config?
> > > >> > > > >
> > > >> > > > > 8) It is mentioned that "To distinguish between previous
> > version
> > > >> of
> > > >> > > > > protocol, we will also increase the join group request
> version
> > > to
> > > >> v4
> > > >> > > when
> > > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> > latest
> > > >> (<
> > > >> > > v4),
> > > >> > > > > the join group request shall be downgraded to v3 without
> > setting
> > > >> the
> > > >> > > > member
> > > >> > > > > Id". It is probably simpler to just say that this feature is
> > > >> enabled
> > > >> > if
> > > >> > > > > JoinGroupRequest V4 is supported on both client and broker
> and
> > > >> > > > MEMBER_NAME
> > > >> > > > > is configured with non-empty string.
> > > >> > > > >
> > > >> > > > > 9) It is mentioned that broker may return
> > > >> NO_STATIC_MEMBER_INFO_SET
> > > >> > > error
> > > >> > > > > in OffsetCommitResponse for "commit requests under static
> > > >> > membership".
> > > >> > > > Can
> > > >> > > > > you clarify how broker determines whether the commit request
> > is
> > > >> under
> > > >> > > > > static membership?
> > > >> > > > >
> > > >> > > > > Thanks,
> > > >> > > > > Dong
> > > >> > > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > --
> > > >> > > > -- Guozhang
> > > >> > > >
> > > >> > >
> > > >> >
> > > >> >
> > > >> > --
> > > >> > -Regards,
> > > >> > Mayuresh R. Gharat
> > > >> > (862) 250-7125
> > > >> >
> > > >>
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Thanks for the replies. Please find the follow up queries below.

    5. Regarding "So in summary, *the member will only be removed due to
session timeout*. We shall remove it from both in-memory static member name
mapping and member list." If the rebalance is invoked manually using the
the admin apis, how long should the group coordinator wait for the members
of the group to send a JoinGroupRequest for participating in the rebalance?
How is a lagging consumer handled?
The plan is to disable member kick out when rebalance.timeout is reached,
so basically we are not "waiting" any
join group request from existing members; we shall just rebalance base on
what we currently have within the group
metadata. Lagging consumer will trigger rebalance later if session timeout
> rebalance timeout.

>
Just wanted to understand this better. Lets take an example, say we have a
> consumer group "GroupA" with 4 consumers  c1, c2, c3, c4.
> Everything is running fine and suddenly C4 host has issues and it goes
> down. Now we notice that we can still operate with c1, c2, c3 and don't
> want to wait for
> c4 to come back up. We use the admin api
> "invokeConsumerRebalance("GroupA")".
> Now the GroupCoordinator, will ask the members c1, c2, c3 to join the
> group again (in there heartBeatResponse) as first step of rebalance.
> Now lets say that c1, c2 immediately send a joinGroupRequest but c3 is
> delayed. At this stage, if we are not "waiting" on any join group request,
> few things can happen :
>
>    - c4's partitions are distributed only among c1,c2. c3 maintains its
>    original assignment. c1, c2 will start processing the newly assigned
>    partitions.
>
> OR
>
>    - c4's partitions are distributed among c1, c2, c3. c1 and c2 start
>    processing the newly assigned partitions. c3 gets to know about the newly
>    assigned partitions later when it sends the JoinGroupRequest (which was
>    delayed).
>
> OR
>
>    - Will the rebalance do a complete reassignment, where c1, c2, c3 have
>    to give up there partitions and all the partitions belonging to c1, c2, c3,
>    c4 will be redistributed among c1, c2, c3 ? If this is the case, the
>    GroupCoordinator needs to give some buffer time for c1, c2, c3 to revoke
>    there partitions and rejoin the group.
>
> This is as per my understanding of how the KIP would work without changing
> the underlying group coordination workflow. Please correct me if I
> misunderstood something here.
>


- When we say that we would use invokeConsumerRebalance(groupId) to down
scale, with the example in the above question, how will the
GroupCoordinator know that c4 should be kicked out of the group since we
are trying to invoke rebalance proactively without waiting for c4's session
time out to expire. Should there be a way of telling the GroupCoordinator
that consumer c4 has been kicked out of the groupId = "GroupA"?

- Also it looks like the statement "If the `member.id` uses
UNKNOWN_MEMBER_NAME, we shall always generate a new member id and replace
the one within current map, if `group.member.name` is known. Also once we
are done with KIP-394
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member.id+for+initial+join+group+request>,
all the join group requests are requiring `member.id` to physically enter
the consumer group. This way the latest joined " is incomplete. Can you
take a look at this?
Also when we say "all the join group requests are requiring `member.id` to
physically enter the consumer group." because a newly started consumer will
not have a "member.id", I assume you mean, once the GroupCoordinator
assigns a member.id to the newly started consumer, it has to use it for any
future JoinGroupRequests. Is my understanding correct?


Thanks,

Mayuresh

On Mon, Nov 26, 2018 at 9:20 PM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Mayuresh and Jason for your follow-ups! Let me try to answer both
> in this reply.
>
>
> >    1. Do you intend to have member.id is a static config like
> member.name
> >    after KIP-345 and KIP-394?
>
> No, we shall only rely on broker to allocate member.id for the consumer
> instances. FYI, I already
>
> started the discussion thread for KIP-394 😊
>
> >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> in
> >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config is
> > set,
> >    we will put it in the initial join group request to identify itself
> as a
> >    static member (static membership); otherwise, we will still send
> >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> (dynamic
> >    membership)."
> >       - What is the value of member_id sent in the first JoinGroupRequest
> >       when member_name is set (using static rebalance)? Is it
> > UNKNOW_MEMBER_ID?
>
> Yes, we could only use unknown member id. Actually this part of the
> proposal is outdated,
>
> let me do another audit of the whole doc. Basically, it is currently
> impossible to send `member.id`
>
> when consumer restarted. Sorry for the confusions!
>
> >    3. Regarding "we are requiring member.id (if not unknown) to match
> the
> >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> case
> >    that if we could have members with the same `member.name` (for
> example
> >    mis-configured instances with a valid member.id but added a used
> member
> >    name on runtime). When member name has duplicates, we could refuse
> join
> >    request from members with an outdated `member.id` (since we update
> the
> >    mapping upon each join group request). In an edge case where the
> client
> >    hits this exception in the response, it is suggesting that some other
> >    consumer takes its spot."
> >       - The part of "some other consumer takes the spot" would be
> >       intentional, right? Also when you say " The edge case that if we
> >       could have members with the same `member.name` (for example
> >       mis-configured instances *with a valid member.id <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> >
> > *but
> >       added a used member name on runtime).", what do you mean by *valid
> >       member id* here? Does it mean that there exist a mapping of
> >       member.name to member.id like *MemberA -> id1* on the
> >       GroupCoordinator and this consumer is trying to join with *
> > member.name
> >       <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0>
> = MemberB and member.id <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0>
> =
> > id1 *
> >       ?
>
> I would take Jason's advice that each time we have unknown member joining
> the group, the broker will
>
> always assign a new and unique id to track its identity. In this way,
> consumer with duplicate member name
>
> will be fenced.
>
> >    4. Depending on your explanation for point 2 and the point 3 above
> >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> >    member_name but unknown member_id, if the consumer sends
> > "UNKNOW_MEMBER_ID"
> >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > give it
> >    a member_id, is the consumer suppose to remember member_id for
> >    joinGroupRequests? If yes, how are restarts handled?
>
> Like explained above, we shall not materialize the member.id. Instead we
> need to rely on broker to allocate
>
> a unique id for consumer just like what we have now.
>
> >    5. Regarding "So in summary, *the member will only be removed due to
> >    session timeout*. We shall remove it from both in-memory static member
> >    name mapping and member list."
> >       - If the rebalance is invoked manually using the the admin apis,
> how
> >       long should the group coordinator wait for the members of the
> > group to send
> >       a JoinGroupRequest for participating in the rebalance? How is a
> > lagging
> >       consumer handled?
>
> The plan is to disable member kick out when rebalance.timeout is reached,
> so basically we are not "waiting" any
>
> join group request from existing members; we shall just rebalance base on
> what we currently have within the group
>
> metadata. Lagging consumer will trigger rebalance later if session timeout
> > rebalance timeout.
>
> >    6. Another detail to take care is that we need to automatically take
> the
> >    hash of group id so that we know which broker to send this request to.
> >       - I assume this should be same as the way we find the coordinator,
> >       today right? If yes, should we specify it in the KIP ?
>
> Yep, it is. Add FindCoordinatorRequest logic to the script.
>
> >    7. Are there any specific failure scenarios when you say "other
> >    potential failure cases."? It would be good to mention them
> explicitly,
> > if
> >    you think there are any.
>
> Nah, I'm gonna remove it because it seems causing more confusion than
> making my assumption clear, which is
>
> "there could be other failure cases that I can't enumerate now" 😊
>
> >    8. It would be good to have a rollback plan as you have for roll
> forward
> >    in the KIP.
>
> Great suggestion! Added a simple rollback plan.
>
>
> Next is answering Jason's suggestions:
>
> 1. This may be the same thing that Mayuresh is asking about. I think the
> suggestion in the KIP is that if a consumer sends JoinGroup with a member
> name, but no member id, then we will return the current member id
> associated with that name. It seems in this case that we wouldn't be able
> to protect from having two consumers active with the same configured
> member.name? For example, imagine that we had a consumer with member.name
> =A
> which is assigned member.id=1. Suppose it becomes a zombie and a new
> instance starts up with member.name=A. If it is also assigned member.id=1,
> then how can we detect the zombie if it comes back to life? Both instances
> will have the same member.id.
>
> The goal is to avoid a rebalance on a rolling restart, but we still need to
> fence previous members. I am wondering if we can generate a new member.id
> every time we receive a request from a static member with an unknown member
> id. If the old instance with the same member.name attempts any operation,
> then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> subscription of the new instance hasn't changed, then we can skip the
> rebalance and return the current assignment without forcing a rebalance.
>
> The trick to making this work is in the error handling of the zombie
> consumer. If the zombie simply resets its member.id and rejoins to get a
> new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> fencing the new member. We want to avoid this. There needs to be an
> expectation for static members that the member.id of a static member will
> not be changed except when a new member with the same member.name joins
> the
> group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> with static member names.
>
> Yep, I like this idea! Keep giving out refresh member.id when facing
> anonymous request will definitely
>
> prevent processing bug due to duplicate consumers, however I don't think I
> fully understand the 3rd paragraph where
>
> you mentioned  "There needs to be an expectation for static members that
> the member.id of a static member will
>
> not be changed except when a new member with the same member.name joins
> the group. "  How do you plan
> to know whether this member is new member or old member? I feel even with
> zombie consumer takes the ownership,
> it should be detected very quickly (as MISMATCH_ID exception trigger
> original consumer instance dies)
> and end user will start to fix it right away. Is there any similar logic
> we applied in fencing duplicate `transaction.id`?
>
> 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far as
> I understand it, it is used for scaling down a consumer group and somehow
> bypasses normal session timeout expiration. I am wondering how critical
> this piece is and whether we can leave it for future work. If not, then it
> would be helpful to elaborate on its implementation. How would the
> coordinator know which members to kick out of the group?
>
> This API is needed when we need to immediately trigger rebalance instead
> of waiting session timeout
>
> or rebalance timeout (Emergent scale up/down). It is very necessary to
> have it for
>
> management purpose because user could choose when to trigger rebalance
> pretty freely,
>
> gaining more client side control.
>
> In the meanwhile I see your point that we need to actually have the
> ability to kick out members that we plan
>
> to scale down fast (as rebalance timeout no longer kicks any offline
> member out of the group), I will think of adding an optional
>
> list of members that are ready to be removed.
>
> Another idea is to let static member send `LeaveGroupRequest` when they
> are going offline (either scale down or bouncing),
>
> and broker will cache this information as "OfflineMembers" without
> triggering rebalance. When handling ConsumerRebalanceRequest broker will
>
> kick the static members that are currently offline and trigger rebalance
> immediately. How does this plan sound?
>
> 3. I've been holding back on mentioning this, but I think we should
> reconsider the name `member.name`. I think we want something that suggests
> its expectation of uniqueness in the group. How about `group.instance.id`
> to go along with `group.id`?
>
> Yea, Dong and Stanislav also mentioned this naming. I personally buy in
> the namespace idea, and
>
> since we already use `member.name` in a lot of context, I decide to
> rename the config to `group.member.name`
>
> which should be sufficient for solving all the concerns we have now.
> Sounds good?
>
>
> Thank you for your great suggestions! Let me know if my reply makes sense
> her.
>
>
> Best,
>
> Boyang
>
> ________________________________
> From: Jason Gustafson <ja...@confluent.io>
> Sent: Tuesday, November 27, 2018 7:51 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for the updates. Looks like we're headed in the right direction and
> clearly the interest that this KIP is receiving shows how strong the
> motivation is!
>
> I have a few questions:
>
> 1. This may be the same thing that Mayuresh is asking about. I think the
> suggestion in the KIP is that if a consumer sends JoinGroup with a member
> name, but no member id, then we will return the current member id
> associated with that name. It seems in this case that we wouldn't be able
> to protect from having two consumers active with the same configured
> member.name? For example, imagine that we had a consumer with member.name
> =A
> which is assigned member.id=1. Suppose it becomes a zombie and a new
> instance starts up with member.name=A. If it is also assigned member.id=1,
> then how can we detect the zombie if it comes back to life? Both instances
> will have the same member.id.
>
> The goal is to avoid a rebalance on a rolling restart, but we still need to
> fence previous members. I am wondering if we can generate a new member.id
> every time we receive a request from a static member with an unknown member
> id. If the old instance with the same member.name attempts any operation,
> then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
> subscription of the new instance hasn't changed, then we can skip the
> rebalance and return the current assignment without forcing a rebalance.
>
> The trick to making this work is in the error handling of the zombie
> consumer. If the zombie simply resets its member.id and rejoins to get a
> new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
> fencing the new member. We want to avoid this. There needs to be an
> expectation for static members that the member.id of a static member will
> not be changed except when a new member with the same member.name joins
> the
> group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
> with static member names.
>
> 2. The mechanics of the ConsumerRebalance API seem unclear to me. As far as
> I understand it, it is used for scaling down a consumer group and somehow
> bypasses normal session timeout expiration. I am wondering how critical
> this piece is and whether we can leave it for future work. If not, then it
> would be helpful to elaborate on its implementation. How would the
> coordinator know which members to kick out of the group?
>
> 3. I've been holding back on mentioning this, but I think we should
> reconsider the name `member.name`. I think we want something that suggests
> its expectation of uniqueness in the group. How about `group.instance.id`
> to go along with `group.id`?
>
> Thanks,
> Jason
>
>
>
> On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Thanks a lot for replying to all the queries and discussions here, so
> > patiently.
> > Really appreciate it.
> >
> > Had a few questions and suggestions after rereading the current version
> of
> > the KIP :
> >
> >
> >    1. Do you intend to have member.id is a static config like
> member.name
> >    after KIP-345 and KIP-394?
> >    2. Regarding "On client side, we add a new config called MEMBER_NAME
> in
> >    ConsumerConfig. On consumer service init, if the MEMBER_NAME config is
> > set,
> >    we will put it in the initial join group request to identify itself
> as a
> >    static member (static membership); otherwise, we will still send
> >    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID
> (dynamic
> >    membership)."
> >       - What is the value of member_id sent in the first JoinGroupRequest
> >       when member_name is set (using static rebalance)? Is it
> > UNKNOW_MEMBER_ID?
> >    3. Regarding "we are requiring member.id (if not unknown) to match
> the
> >    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge
> case
> >    that if we could have members with the same `member.name` (for
> example
> >    mis-configured instances with a valid member.id but added a used
> member
> >    name on runtime). When member name has duplicates, we could refuse
> join
> >    request from members with an outdated `member.id` (since we update
> the
> >    mapping upon each join group request). In an edge case where the
> client
> >    hits this exception in the response, it is suggesting that some other
> >    consumer takes its spot."
> >       - The part of "some other consumer takes the spot" would be
> >       intentional, right? Also when you say " The edge case that if we
> >       could have members with the same `member.name` (for example
> >       mis-configured instances *with a valid member.id <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0
> >
> > *but
> >       added a used member name on runtime).", what do you mean by *valid
> >       member id* here? Does it mean that there exist a mapping of
> >       member.name to member.id like *MemberA -> id1* on the
> >       GroupCoordinator and this consumer is trying to join with *
> > member.name
> >       <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0>
> = MemberB and member.id <
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0>
> =
> > id1 *
> >       ?
> >    4. Depending on your explanation for point 2 and the point 3 above
> >    regarding returning back MEMBER_ID_MISMATCH on having a matching
> >    member_name but unknown member_id, if the consumer sends
> > "UNKNOW_MEMBER_ID"
> >    on the first JoinGroupRequest and relies on the GroupCoordinator to
> > give it
> >    a member_id, is the consumer suppose to remember member_id for
> >    joinGroupRequests? If yes, how are restarts handled?
> >    5. Regarding "So in summary, *the member will only be removed due to
> >    session timeout*. We shall remove it from both in-memory static member
> >    name mapping and member list."
> >       - If the rebalance is invoked manually using the the admin apis,
> how
> >       long should the group coordinator wait for the members of the
> > group to send
> >       a JoinGroupRequest for participating in the rebalance? How is a
> > lagging
> >       consumer handled?
> >    6. Another detail to take care is that we need to automatically take
> the
> >    hash of group id so that we know which broker to send this request to.
> >       - I assume this should be same as the way we find the coordinator,
> >       today right? If yes, should we specify it in the KIP ?
> >    7. Are there any specific failure scenarios when you say "other
> >    potential failure cases."? It would be good to mention them
> explicitly,
> > if
> >    you think there are any.
> >    8. It would be good to have a rollback plan as you have for roll
> forward
> >    in the KIP.
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> > gharatmayuresh15@gmail.com>
> > wrote:
> >
> > > Hi Boyang,
> > >
> > > Do you have a discuss thread for KIP-394 that you mentioned here ?
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > >> Hey Dong, thanks for the follow-up here!
> > >>
> > >>
> > >> 1) It is not very clear to the user what is the difference between
> > >> member.name and client.id as both seems to be used to identify the
> > >> consumer. I am wondering if it would be more intuitive to name it
> > >> group.member.name (preferred choice since it matches the current
> > group.id
> > >> config name) or rebalance.member.name to explicitly show that the id
> is
> > >> solely used for rebalance.
> > >> Great question. I feel `member.name` is enough to explain itself, it
> > >> seems not very
> > >> helpful to make the config name longer. Comparing `name` with `id`
> gives
> > >> user the
> > >> impression that they have the control over it with customized rule
> than
> > >> library decided.
> > >>
> > >> 2) In the interface change section it is said that
> > >> GroupMaxSessionTimeoutMs
> > >> will be changed to 30 minutes. It seems to suggest that we will change
> > the
> > >> default value of this config. It does not seem necessary to increase
> the
> > >> time of consumer failure detection when user doesn't use static
> > >> membership.
> > >> Also, say static membership is enabled, then this default config
> change
> > >> will cause a partition to be unavailable for consumption for 30
> minutes
> > if
> > >> there is hard consumer failure, which seems to be worse experience
> than
> > >> having unnecessary rebalance (when this timeout is small),
> particularly
> > >> for
> > >> new users of Kafka. Could you explain more why we should make this
> > change?
> > >> We are not changing the default session timeout value. We are just
> > >> changing the
> > >> cap we are enforcing on the session timeout max value. So this change
> is
> > >> not affecting
> > >> what kind of membership end user is using, and loosing the cap is
> giving
> > >> end user
> > >> more flexibility on trade-off between liveness and stability.
> > >>
> > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> DUPLICATE_STATIC_MEMBER
> > >> into one error? It seems that these two errors are currently handled
> by
> > >> the
> > >> consumer in the same way. And we don't also don't expect
> > >> MEMBER_ID_MISMATCH
> > >> to happen. Thus it is not clear what is the benefit of having two
> > errors.
> > >> I agree that we should remove DUPLICATE_STATIC_MEMBER error because
> with
> > >> the KIP-394<
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=EDM7PmpOo2HenYhFHX2rxrszpkI7di401WhKh2Vjw5k%3D&amp;reserved=0
> > >> >
> > >> we will automatically fence all join requests with UNKNOWN_MEMBER_ID.
> > >>
> > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > contains
> > >> member name which is already in the consumer group, however the member
> > id
> > >> was missing". After a consumer is restarted, it will send a
> > >> JoinGroupRequest with an existing memberName (as the coordinator has
> not
> > >> expired this member from the memory) and memberId
> > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> > >> across consumer restart in the consumer side). Does it mean that
> > >> JoinGroupRequest from a newly restarted consumer will always be
> rejected
> > >> until the sessionTimeoutMs has passed?
> > >> Same answer as question 3). This part of the logic shall be removed
> from
> > >> the proposal.
> > >>
> > >> 5) It seems that we always add two methods to the interface
> > >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> > the
> > >> other without option. Could this be specified in the interface change
> > >> section?
> > >> Sounds good! Added both methods.
> > >>
> > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> trigger
> > >> rebalance? If so, we probably want to specify the command line tool
> > >> interface similar to
> > >>
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> > >> .
> > >> Added the script.
> > >>
> > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > meaning
> > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > >> simpler
> > >> to allows this API to trigger rebalance regardless of whether consumer
> > is
> > >> configured with memberName.
> > >> Sounds good. Right now I feel for both static and dynamic membership
> it
> > is
> > >> more manageable to introduce the consumer rebalance method through
> admin
> > >> client API.
> > >>
> > >> 8) It is not very clear how the newly added AdminClient API trigger
> > >> rebalance. For example, does it send request? Can this be explained in
> > the
> > >> KIP?
> > >>
> > >> Sure, I will add more details to the API.
> > >>
> > >>
> > >> Thanks again for the helpful suggestions!
> > >>
> > >>
> > >> Best,
> > >> Boyang
> > >>
> > >> ________________________________
> > >> From: Dong Lin <li...@gmail.com>
> > >> Sent: Saturday, November 24, 2018 2:54 PM
> > >> To: dev
> > >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > >> specifying member id
> > >>
> > >> Hey Boyang,
> > >>
> > >> Thanks for the update! Here are some followup comments:
> > >>
> > >> 1) It is not very clear to the user what is the difference between
> > >> member.name and client.id as both seems to be used to identify the
> > >> consumer. I am wondering if it would be more intuitive to name it
> > >> group.member.name (preferred choice since it matches the current
> > group.id
> > >> config name) or rebalance.member.name to explicitly show that the id
> is
> > >> solely used for rebalance.
> > >>
> > >> 2) In the interface change section it is said that
> > >> GroupMaxSessionTimeoutMs
> > >> will be changed to 30 minutes. It seems to suggest that we will change
> > the
> > >> default value of this config. It does not seem necessary to increase
> the
> > >> time of consumer failure detection when user doesn't use static
> > >> membership.
> > >> Also, say static membership is enabled, then this default config
> change
> > >> will cause a partition to be unavailable for consumption for 30
> minutes
> > if
> > >> there is hard consumer failure, which seems to be worse experience
> than
> > >> having unnecessary rebalance (when this timeout is small),
> particularly
> > >> for
> > >> new users of Kafka. Could you explain more why we should make this
> > change?
> > >>
> > >> 3) Could we just combine MEMBER_ID_MISMATCH and
> DUPLICATE_STATIC_MEMBER
> > >> into one error? It seems that these two errors are currently handled
> by
> > >> the
> > >> consumer in the same way. And we don't also don't expect
> > >> MEMBER_ID_MISMATCH
> > >> to happen. Thus it is not clear what is the benefit of having two
> > errors.
> > >>
> > >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> > contains
> > >> member name which is already in the consumer group, however the member
> > id
> > >> was missing". After a consumer is restarted, it will send a
> > >> JoinGroupRequest with an existing memberName (as the coordinator has
> not
> > >> expired this member from the memory) and memberId
> > >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> > >> across consumer restart in the consumer side). Does it mean that
> > >> JoinGroupRequest from a newly restarted consumer will always be
> rejected
> > >> until the sessionTimeoutMs has passed?
> > >>
> > >> 5) It seems that we always add two methods to the interface
> > >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> > the
> > >> other without option. Could this be specified in the interface change
> > >> section?
> > >>
> > >> 6) Do we plan to have off-the-shelf command line tool for SRE to
> trigger
> > >> rebalance? If so, we probably want to specify the command line tool
> > >> interface similar to
> > >>
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> > >> .
> > >>
> > >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> > >> "invokeConsumerRebalance"? It is not very clear what is the extra
> > meaning
> > >> of world "force" as compared to "trigger" or "invoke". And it seems
> > >> simpler
> > >> to allows this API to trigger rebalance regardless of whether consumer
> > is
> > >> configured with memberName.
> > >>
> > >> 8) It is not very clear how the newly added AdminClient API trigger
> > >> rebalance. For example, does it send request? Can this be explained in
> > the
> > >> KIP?
> > >>
> > >> Thanks,
> > >> Dong
> > >>
> > >>
> > >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> > wrote:
> > >>
> > >> > Hey Mayuresh,
> > >> >
> > >> >
> > >> > thanks for your feedbacks! I will try do another checklist here.
> > >> >
> > >> >
> > >> > > By this you mean, even if the application has not called
> > >> > > KafkaConsumer.poll() within session timeout, it will not be
> sending
> > >> the
> > >> > > LeaveGroup request, right?
> > >> >
> > >> > Yep it's true, we will prevent client from sending leave group
> request
> > >> > when they are set with `member.name`.
> > >> >
> > >> >
> > >> > > When is the member.name removed from this map?
> > >> > Good question, we will only kick off member due to session timeout
> > >> within
> > >> > static membership. Let me update the KIP to clearly assert that.
> > >> >
> > >> > > How is this case (missing member id) handled on the client side?
> > What
> > >> is
> > >> > the application that
> > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > >> > I have extended the two exceptions within join group response V4.
> > >> > Basically I define both corresponding actions to be immediate
> failing
> > >> > client application, because so far it is unknown what kind of client
> > >> issue
> > >> > could trigger them. After the first version, we will keep enhance
> the
> > >> error
> > >> > handling logic!
> > >> >
> > >> > > This would mean that it might take more time to detect unowned
> topic
> > >> > > partitions and may cause delay for applications that perform data
> > >> > mirroring
> > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > make
> > >> > here
> > >> > > as listed below separately.
> > >> > The goal of extending session timeout cap is for users with good
> > client
> > >> > side monitoring tools that could auto-heal the dead consumers very
> > >> fast. So
> > >> > it is optional (and personal) to extend session timeout to a
> > reasonable
> > >> > number with different client scenarios.
> > >> >
> > >> > > you meant remove unjoined members of the group, right ?
> > >> > Yep, there is a typo. Thanks for catching this!
> > >> >
> > >> > > What do you mean by " Internally we would optimize this logic by
> > >> having
> > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > stage,
> > >> > > without removing non-responsive members immediately." There would
> > not
> > >> be
> > >> > a
> > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > later,
> > >> > > right ? If yes, can you highlight this in the KIP ?
> > >> > No, there won't be. We want to limit the rebalance timeout
> > functionality
> > >> > to only use as a timer to
> > >> > end prepare rebalance stage. This way, late joining static members
> > will
> > >> > not trigger further rebalance
> > >> > as long as they are within session timeout. I added your highlight
> to
> > >> the
> > >> > KIP!
> > >> >
> > >> > > The KIP talks about scale up scenario but its not quite clear how
> we
> > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> adding
> > >> > status
> > >> > > "learner" ?. Can you shed more light on how this is handled in the
> > >> KIP,
> > >> > if
> > >> > > its handled?
> > >> > Updated the KIP: we shall not cover scale up case in 345, because we
> > >> > believe client side could
> > >> > better handle this logic.
> > >> >
> > >> > > I think Jason had brought this up earlier about having a way to
> say
> > >> how
> > >> > > many members/consumer hosts are you choosing to be in the consumer
> > >> group.
> > >> > > If we can do this, then in case of mirroring applications we can
> do
> > >> this
> > >> > :
> > >> > > Lets say we have a mirroring application that consumes from Kafka
> > >> cluster
> > >> > > A and produces to Kafka cluster B.
> > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > >> service
> > >> > > providers can set a mirroring group saying that it will take, for
> > >> example
> > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > >> latency
> > >> > > for mirroring and can have additional 10 consumer hosts as spare
> in
> > >> the
> > >> > > same group.
> > >> > > So when the first 300 members/consumers to join the group will
> start
> > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > >> > > The remaining 10 consumer members can sit idle.
> > >> > > The moment one of the consumer (for example: consumer number 54)
> > from
> > >> the
> > >> > > first 300 members go out of the group (crossed session timeout),
> it
> > >> (the
> > >> > > groupCoordinator) can just assign the topicPartitions from the
> > >> consumer
> > >> > > member 54 to one of the spare hosts.
> > >> > > Once the consumer member 54 comes back up, it can start as being a
> > >> part
> > >> > of
> > >> > > the spare pool.
> > >> > > This enables us to have lower session timeouts and low latency
> > >> mirroring,
> > >> > > in cases where the service providers are OK with having spare
> hosts.
> > >> > > This would mean that we would tolerate n consumer members leaving
> > and
> > >> > > rejoining the group and still provide low latency as long as n <=
> > >> number
> > >> > of
> > >> > > spare consumers.
> > >> > > If there are no spare host available, we can get back to the idea
> as
> > >> > > described in the KIP.
> > >> > Great idea! In fact on top of static membership we could later
> > introduce
> > >> > APIs to set hard-coded
> > >> > client ids to the group and replace the dead host, or as you
> proposed
> > to
> > >> > define spare host as
> > >> > what I understood as hot backup. I will put both Jason and your
> > >> > suggestions into a separate section
> > >> > called "Future works". Note that this spare host idea may be also
> > >> solvable
> > >> > through rebalance protocol
> > >> > IMO.
> > >> >
> > >> > Thank you again for the great feedback!
> > >> >
> > >> > Boyang
> > >> > ________________________________
> > >> > From: Boyang Chen <bc...@outlook.com>
> > >> > Sent: Thursday, November 22, 2018 3:39 PM
> > >> > To: dev@kafka.apache.org
> > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > >> > specifying member id
> > >> >
> > >> > Hey Dong, sorry for missing your message. I couldn't find your email
> > on
> > >> my
> > >> > thread, so I will just do a checklist here!
> > >> >
> > >> >
> > >> > 1) The motivation currently explicitly states that the goal is to
> > >> improve
> > >> >
> > >> > performance for heavy state application. It seems that the
> motivation
> > >> can
> > >> >
> > >> > be stronger with the following use-case. Currently for MirrorMaker
> > >> cluster
> > >> >
> > >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> > rolling
> > >> >
> > >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > restart
> > >> >
> > >> > will trigger a rebalance which currently pause the consumption of
> the
> > >> all
> > >> >
> > >> > partitions of the MirrorMaker cluster. With the change stated in
> this
> > >> >
> > >> > patch, as long as a MirrorMaker can restart within the specified
> > timeout
> > >> >
> > >> > (e.g. 2 minutes), then we only need constant number of rebalance
> (e.g.
> > >> for
> > >> >
> > >> > leader restart) for the entire rolling bounce, which will
> > significantly
> > >> >
> > >> > improves the availability of the MirrorMaker pipeline. In my
> opinion,
> > >> the
> > >> >
> > >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> > >> consumer
> > >> >
> > >> > process can be restarted within soon, which helps performance even
> if
> > >> >
> > >> > overhead of state shuffling for a given process is small.
> > >> >
> > >> > I just rephrased this part and added it to the KIP. Thanks for
> making
> > >> the
> > >> > motivation more solid!
> > >> >
> > >> > 2) In order to simplify the KIP reading, can you follow the writeup
> > >> style
> > >> > of other KIP (e.g. KIP-98) and list the interface change such as new
> > >> > configs (e.g. registration timeout), new request/response, new
> > >> AdminClient
> > >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> some
> > of
> > >> > these are specified in the Proposed Change section which makes it a
> > bit
> > >> > inconvenient to understand the new interface that will be exposed to
> > >> user.
> > >> > Explanation of the current two-phase rebalance protocol probably can
> > be
> > >> > moved out of public interface section.
> > >> > This is a great suggestion! I just consolidated all the public API
> > >> > changes, and the whole KIP
> > >> > looks much more organized!
> > >> >
> > >> > 3) There are currently two version of JoinGroupRequest in the KIP
> and
> > >> only
> > >> > one of them has field memberId. This seems confusing.
> > >> > Yep, I already found this issue and fixed it.
> > >> >
> > >> > 4) It is mentioned in the KIP that "An admin API to force rebalance
> > >> could
> > >> > be helpful here, but we will make a call once we finished the major
> > >> > implementation". So this seems to be still an open question in the
> > >> current
> > >> > design. We probably want to agree on this before voting for the KIP.
> > >> > We have finalized the idea that this API is needed.
> > >> >
> > >> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> you
> > >> > specify the name of the config key and the default config value?
> > >> Possible
> > >> > default values include empty string or null (similar to
> > transaction.id<
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=N8RxV6%2Bh7ib9CMpW3ZyFq3m2awY1sRPHzlOTi6qU5XY%3D&amp;reserved=0
> > >> >
> > >> > in
> > >> > producer config).
> > >> > I have defined the `member.name` in "New configuration" section.
> > >> >
> > >> > 6) Regarding the use of the topic "static_member_map" to persist
> > member
> > >> > name map, currently if consumer coordinator broker goes offline,
> > >> rebalance
> > >> > is triggered and consumers will try connect to the new coordinator.
> If
> > >> > these consumers can connect to the new coordinator within
> > >> > max.poll.interval.ms<
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> > >> >
> > >> > which by default is 5 minutes, given that broker can
> > >> > use a deterministic algorithm to determine the partition ->
> > member_name
> > >> > mapping, each consumer should get assigned the same set of
> partitions
> > >> > without requiring state shuffling. So it is not clear whether we
> have
> > a
> > >> > strong use-case for this new logic. Can you help clarify what is the
> > >> > benefit of using topic "static_member_map" to persist member name
> map?
> > >> > I have discussed with Guozhang offline, and I believe reusing the
> > >> current
> > >> > `_consumer_offsets`
> > >> > topic is a better and unified solution.
> > >> >
> > >> > 7) Regarding the introduction of the expensionTimeoutMs config, it
> is
> > >> > mentioned that "we are using expansion timeout to replace rebalance
> > >> > timeout, which is configured by max.poll.intervals from client side,
> > and
> > >> > using registration timeout to replace session timeout". Currently
> the
> > >> > default max.poll.interval.ms<
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> > >> >
> > >> > is configured to be 5 minutes and there will
> > >> > be only one rebalance if all new consumers can join within 5
> minutes.
> > >> So it
> > >> > is not clear whether we have a strong use-case for this new config.
> > Can
> > >> you
> > >> > explain what is the benefit of introducing this new config?
> > >> > Previously our goal is to use expansion timeout as a workaround for
> > >> > triggering multiple
> > >> > rebalances when scaling up members are not joining at the same time.
> > It
> > >> is
> > >> > decided to
> > >> > be addressed by client side protocol change, so we will not
> introduce
> > >> > expansion timeout.
> > >> >
> > >> > 8) It is mentioned that "To distinguish between previous version of
> > >> > protocol, we will also increase the join group request version to v4
> > >> when
> > >> > MEMBER_NAME is set" and "If the broker version is not the latest (<
> > v4),
> > >> > the join group request shall be downgraded to v3 without setting the
> > >> member
> > >> > Id". It is probably simpler to just say that this feature is enabled
> > if
> > >> > JoinGroupRequest V4 is supported on both client and broker and
> > >> MEMBER_NAME
> > >> > is configured with non-empty string.
> > >> > Yep, addressed this!
> > >> >
> > >> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> > >> error
> > >> > in OffsetCommitResponse for "commit requests under static
> membership".
> > >> Can
> > >> > you clarify how broker determines whether the commit request is
> under
> > >> > static membership?
> > >> >
> > >> > We have agreed that commit request shouldn't be affected by the new
> > >> > membership, thus
> > >> > removing it here. Thanks for catching this!
> > >> >
> > >> > Let me know if you have further suggestions or concerns. Thank you
> for
> > >> > your valuable feedback
> > >> > to help me design the KIP better! (And I will try to address your
> > >> > feedbacks in next round Mayuresh ??)
> > >> >
> > >> > Best,
> > >> > Boyang
> > >> > ________________________________
> > >> > From: Mayuresh Gharat <gh...@gmail.com>
> > >> > Sent: Wednesday, November 21, 2018 7:50 AM
> > >> > To: dev@kafka.apache.org
> > >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > >> > specifying member id
> > >> >
> > >> > Hi Boyang,
> > >> >
> > >> > Thanks for updating the KIP. This is a step good direction for
> > stateful
> > >> > applications and also mirroring applications whose latency is
> affected
> > >> due
> > >> > to the rebalance issues that we have today.
> > >> >
> > >> > I had a few questions on the current version of the KIP :
> > >> > For the effectiveness of the KIP, consumer with member.name set
> will
> > >> *not
> > >> > send leave group request* when they go offline
> > >> >
> > >> > > By this you mean, even if the application has not called
> > >> > > KafkaConsumer.poll() within session timeout, it will not be
> sending
> > >> the
> > >> > > LeaveGroup request, right?
> > >> > >
> > >> >
> > >> > Broker will maintain an in-memory mapping of {member.name ?
> member.id
> > }
> > >> to
> > >> > track member uniqueness.
> > >> >
> > >> > > When is the member.name removed from this map?
> > >> > >
> > >> >
> > >> > Member.id must be set if the *member.name <
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0
> > >> >
> > >> > *is already
> > >> > within the map. Otherwise reply MISSING_MEMBER_ID
> > >> >
> > >> > > How is this case handled on the client side? What is the
> application
> > >> that
> > >> > > is using the KafkaConsumer suppose to do in this scenario?
> > >> > >
> > >> >
> > >> > Session timeout is the timeout we will trigger rebalance when a
> member
> > >> goes
> > >> > offline for too long (not sending heartbeat request). To make static
> > >> > membership effective, we should increase the default max session
> > >> timeout to
> > >> > 30 min so that end user could config it freely.
> > >> >
> > >> > > This would mean that it might take more time to detect unowned
> topic
> > >> > > partitions and may cause delay for applications that perform data
> > >> > mirroring
> > >> > > tasks. I discussed this with our sre and we have a suggestion to
> > make
> > >> > here
> > >> > > as listed below separately.
> > >> > >
> > >> >
> > >> > Currently there is a config called *rebalance timeout* which is
> > >> configured
> > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > interval
> > >> is
> > >> > because consumer could only send request within the call of poll()
> and
> > >> we
> > >> > want to wait sufficient time for the join group request. When
> reaching
> > >> > rebalance timeout, the group will move towards completingRebalance
> > stage
> > >> > and remove unjoined groups
> > >> >
> > >> > > you meant remove unjoined members of the group, right ?
> > >> > >
> > >> >
> > >> > Currently there is a config called *rebalance timeout* which is
> > >> configured
> > >> > by consumer *max.poll.intervals*. The reason we set it to poll
> > interval
> > >> is
> > >> > because consumer could only send request within the call of poll()
> and
> > >> we
> > >> > want to wait sufficient time for the join group request. When
> reaching
> > >> > rebalance timeout, the group will move towards completingRebalance
> > stage
> > >> > and remove unjoined groups. This is actually conflicting with the
> > >> design of
> > >> > static membership, because those temporarily unavailable members
> will
> > >> > potentially reattempt the join group and trigger extra rebalances.
> > >> > Internally we would optimize this logic by having rebalance timeout
> > >> only in
> > >> > charge of stopping prepare rebalance stage, without removing
> > >> non-responsive
> > >> > members immediately.
> > >> >
> > >> > > What do you mean by " Internally we would optimize this logic by
> > >> having
> > >> > > rebalance timeout only in charge of stopping prepare rebalance
> > stage,
> > >> > > without removing non-responsive members immediately." There would
> > not
> > >> be
> > >> > a
> > >> > > full rebalance if the lagging consumer sent a JoinGroup request
> > later,
> > >> > > right ? If yes, can you highlight this in the KIP ?
> > >> > >
> > >> >
> > >> > Scale Up
> > >> >
> > >> > > The KIP talks about scale up scenario but its not quite clear how
> we
> > >> > > handle it. Are we adding a separate "expansion.timeout" or we
> adding
> > >> > status
> > >> > > "learner" ?. Can you shed more light on how this is handled in the
> > >> KIP,
> > >> > if
> > >> > > its handled?
> > >> > >
> > >> >
> > >> >
> > >> > *Discussion*
> > >> > Larger session timeouts causing latency rise for getting data for
> > >> un-owned
> > >> > topic partitions :
> > >> >
> > >> > > I think Jason had brought this up earlier about having a way to
> say
> > >> how
> > >> > > many members/consumer hosts are you choosing to be in the consumer
> > >> group.
> > >> > > If we can do this, then in case of mirroring applications we can
> do
> > >> this
> > >> > :
> > >> > > Lets say we have a mirroring application that consumes from Kafka
> > >> cluster
> > >> > > A and produces to Kafka cluster B.
> > >> > > Depending on the data and the Kafka cluster configuration, Kafka
> > >> service
> > >> > > providers can set a mirroring group saying that it will take, for
> > >> example
> > >> > > 300 consumer hosts/members to achieve the desired throughput and
> > >> latency
> > >> > > for mirroring and can have additional 10 consumer hosts as spare
> in
> > >> the
> > >> > > same group.
> > >> > > So when the first 300 members/consumers to join the group will
> start
> > >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > >> > > The remaining 10 consumer members can sit idle.
> > >> > > The moment one of the consumer (for example: consumer number 54)
> > from
> > >> the
> > >> > > first 300 members go out of the group (crossed session timeout),
> it
> > >> (the
> > >> > > groupCoordinator) can just assign the topicPartitions from the
> > >> consumer
> > >> > > member 54 to one of the spare hosts.
> > >> > > Once the consumer member 54 comes back up, it can start as being a
> > >> part
> > >> > of
> > >> > > the spare pool.
> > >> > > This enables us to have lower session timeouts and low latency
> > >> mirroring,
> > >> > > in cases where the service providers are OK with having spare
> hosts.
> > >> > > This would mean that we would tolerate n consumer members leaving
> > and
> > >> > > rejoining the group and still provide low latency as long as n <=
> > >> number
> > >> > of
> > >> > > spare consumers.
> > >> > > If there are no spare host available, we can get back to the idea
> as
> > >> > > described in the KIP.
> > >> > >
> > >> >
> > >> > Thanks,
> > >> >
> > >> > Mayuresh
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > >> > konstantine@confluent.io> wrote:
> > >> >
> > >> > > Hi Boyang.
> > >> > >
> > >> > > Thanks for preparing this KIP! It is making good progress and will
> > be
> > >> a
> > >> > > great improvement for stateful Kafka applications.
> > >> > >
> > >> > > Apologies for my late reply, I was away for a while. Lots of great
> > >> > comments
> > >> > > so far, so I'll probably second most of them in what I suggest
> below
> > >> at
> > >> > > this point.
> > >> > >
> > >> > > When I first read the KIP, I wanted to start at the end with
> > something
> > >> > that
> > >> > > wasn't highlighted a lot. That was the topic related to handling
> > >> > duplicate
> > >> > > members. I see now that the initial suggestion of handling this
> > >> situation
> > >> > > during offset commit has been removed, and I agree with that.
> Issues
> > >> > > related to membership seem to be handled better when the member
> > joins
> > >> the
> > >> > > group rather than when it tries to commit offsets. This also
> > >> simplifies
> > >> > how
> > >> > > many request types need to change in order to incorporate the new
> > >> member
> > >> > > name field.
> > >> > >
> > >> > > I also agree with what Jason and Guozhang have said regarding
> > >> timeouts.
> > >> > > Although semantically, it's easier to think of every operation
> > having
> > >> its
> > >> > > own timeout, operationally this can become a burden. Thus,
> > >> consolidation
> > >> > > seems preferable here. The definition of embedded protocols on top
> > of
> > >> the
> > >> > > base group membership protocol for rebalancing gives enough
> > >> flexibility
> > >> > to
> > >> > > address such needs in each client component separately.
> > >> > >
> > >> > > Finally, some minor comments:
> > >> > > In a few places the new/proposed changes are referred to as
> > "current".
> > >> > > Which is a bit confusing considering that there is a protocol in
> > place
> > >> > > already, and by "current" someone might understand the existing
> one.
> > >> I'd
> > >> > > recommend using new/proposed or equivalent when referring to
> changes
> > >> > > introduced with KIP-345 and current/existing or equivalent when
> > >> referring
> > >> > > to existing behavior.
> > >> > >
> > >> > > There's the following sentence in the "Public Interfaces" section:
> > >> > > "Since for many stateful consumer/stream applications, the state
> > >> > shuffling
> > >> > > is more painful than short time partial unavailability."
> > >> > > However, my understanding is that the changes proposed with
> KIP-345
> > >> will
> > >> > > not exploit any partial availability. A suggestion for dealing
> with
> > >> > > temporary imbalances has been made in "Incremental Cooperative
> > >> > Rebalancing"
> > >> > > which can work well with KIP-345, but here I don't see proposed
> > >> changes
> > >> > > that suggest that some resources (e.g. partitions) will keep being
> > >> used
> > >> > > while others will not be utilized. Thus, you might want to adjust
> > this
> > >> > > sentence. Correct me if I'm missing something related to that.
> > >> > >
> > >> > > In the rejected alternatives, under point 2) I read "we can copy
> the
> > >> > member
> > >> > > id to the config files". I believe it means to say "member name"
> > >> unless
> > >> > I'm
> > >> > > missing something about reusing member ids. Also below I read: "By
> > >> > allowing
> > >> > > consumers to optionally specifying a member id" which probably
> > implies
> > >> > > "member name" again. In a sense this section highlights a
> potential
> > >> > > confusion between member name and member id. I wonder if we could
> > >> come up
> > >> > > with a better term for the new field. StaticTag, StaticLabel, or
> > even
> > >> > > StaticName are some suggestions that could potentially help with
> > >> > confusion
> > >> > > between MemberId and MemberName and what corresponds to what. But
> I
> > >> > > wouldn't like to disrupt the discussion with naming conventions
> too
> > >> much
> > >> > at
> > >> > > this point. I just mention it here as a thought.
> > >> > >
> > >> > > Looking forward to see the final details of this KIP. Great work
> so
> > >> far!
> > >> > >
> > >> > > Konstantine
> > >> > >
> > >> > >
> > >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com>
> > >> wrote:
> > >> > >
> > >> > > > Thanks Guozhang for the great summary here, and I have been
> > >> following
> > >> > up
> > >> > > > the action items here.
> > >> > > >
> > >> > > >
> > >> > > >   1.  I already updated the KIP to remove the expansion timeout
> > and
> > >> > > > registration timeout. Great to see them being addressed in
> client
> > >> side!
> > >> > > >   2.  I double checked the design and I believe that it is ok to
> > >> have
> > >> > > both
> > >> > > > static member and dynamic member co-exist in the same group. So
> > the
> > >> > > upgrade
> > >> > > > shouldn't be destructive and we are removing the two membership
> > >> > protocol
> > >> > > > switching APIs.
> > >> > > >   3.  I only have question about this one. I'm still reading the
> > >> > > KafkaApis
> > >> > > > code here. Should I just use the same authorization logic for
> > >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > >> > > >   4.  I'm very excited to see this work with K8! Like you
> > suggested,
> > >> > this
> > >> > > > feature could be better addressed in a separate KIP because it
> is
> > >> > pretty
> > >> > > > independent. I could start drafting the KIP once the current
> > >> proposal
> > >> > is
> > >> > > > approved.
> > >> > > >   5.  I believe that we don't need fencing in offset commit
> > request,
> > >> > > since
> > >> > > > duplicate member.name issue could be handled by join group
> > >> request. We
> > >> > > > shall reject join group with known member name but no member id
> > >> (which
> > >> > > > means we already have an active member using this identity).
> > >> > > >   6.  I agree to remove that internal config once we move
> forward
> > >> with
> > >> > > > static membership. And I already removed the entire section from
> > the
> > >> > KIP.
> > >> > > >
> > >> > > > Let me know if you have other concerns.
> > >> > > >
> > >> > > > Best,
> > >> > > > Boyang
> > >> > > > ________________________________
> > >> > > > From: Guozhang Wang <wa...@gmail.com>
> > >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > >> > > > To: dev
> > >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > >> by
> > >> > > > specifying member id
> > >> > > >
> > >> > > > Hello Boyang,
> > >> > > >
> > >> > > > Thanks a lot for the KIP! It is a great write-up and I
> appreciate
> > >> your
> > >> > > > patience answering to the feedbacks from the community. I'd like
> > to
> > >> add
> > >> > > my
> > >> > > > 2cents here:
> > >> > > >
> > >> > > > 1. By introducing another two timeout configs,
> > registration_timeout
> > >> and
> > >> > > > expansion_timeout, we are effectively having four timeout
> configs:
> > >> > > session
> > >> > > > timeout, rebalance timeout (configured as "max.poll.interval.ms
> "
> > on
> > >> > > client
> > >> > > > side), and these two. Interplaying these timeout configs can be
> > >> quite
> > >> > > hard
> > >> > > > for users with such complexity, and hence I'm wondering if we
> can
> > >> > > simplify
> > >> > > > the situation with as less possible timeout configs as possible.
> > >> Here
> > >> > is
> > >> > > a
> > >> > > > concrete suggestion I'd like propose:
> > >> > > >
> > >> > > > 1.a) Instead of introducing a registration_timeout in addition
> to
> > >> the
> > >> > > > session_timeout for static members, we can just reuse the
> > >> > session_timeout
> > >> > > > and ask users to set it to a larger value when they are
> upgrading
> > a
> > >> > > dynamic
> > >> > > > client to a static client by setting the "member.name" at the
> > same
> > >> > time.
> > >> > > > By
> > >> > > > default, the broker-side min.session.timeout is 6 seconds and
> > >> > > > max.session.timeout is 5 minutes, which seems reasonable to me
> (we
> > >> can
> > >> > of
> > >> > > > course modify this broker config to enlarge the valid interval
> if
> > we
> > >> > want
> > >> > > > in practice). And then we should also consider removing the
> > >> condition
> > >> > for
> > >> > > > marking a client as failed if the rebalance timeout has reached
> > >> while
> > >> > the
> > >> > > > JoinGroup was not received, so that the semantics of
> > session_timeout
> > >> > and
> > >> > > > rebalance_timeout are totally separated: the former is only used
> > to
> > >> > > > determine if a consumer member of the group should be marked as
> > >> failed
> > >> > > and
> > >> > > > kicked out of the group, and the latter is only used to
> determine
> > >> the
> > >> > > > longest time coordinator should wait for PREPARE_REBALANCE
> phase.
> > In
> > >> > > other
> > >> > > > words if a member did not send the JoinGroup in time of the
> > >> > > > rebalance_timeout, we still include it in the new generation of
> > the
> > >> > group
> > >> > > > and use its old subscription info to send to leader for
> > assignment.
> > >> > Later
> > >> > > > if the member came back with HeartBeat request, we can still
> > follow
> > >> the
> > >> > > > normal path to bring it to the latest generation while checking
> > that
> > >> > its
> > >> > > > sent JoinGroup request contains the same subscription info as we
> > >> used
> > >> > to
> > >> > > > assign the partitions previously (which should be likely the
> case
> > in
> > >> > > > practice). In addition, we should let static members to not send
> > the
> > >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> > static
> > >> > > member
> > >> > > > can only be leaving the group if its session has timed out, OR
> it
> > >> has
> > >> > > been
> > >> > > > indicated to not exist in the group any more (details below).
> > >> > > >
> > >> > > > 1.b) We have a parallel discussion about Incremental Cooperative
> > >> > > > Rebalancing, in which we will encode the "when to rebalance"
> logic
> > >> at
> > >> > the
> > >> > > > application level, instead of at the protocol level. By doing
> this
> > >> we
> > >> > can
> > >> > > > also enable a few other optimizations, e.g. at the Streams level
> > to
> > >> > first
> > >> > > > build up the state store as standby tasks and then trigger a
> > second
> > >> > > > rebalance to actually migrate the active tasks while keeping the
> > >> actual
> > >> > > > rebalance latency and hence unavailability window to be small (
> > >> > > >
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=OAmsz8pz4JW%2BayjLqwk04E16G%2FTCF%2BbVk0LNB%2BUJgeY%3D&amp;reserved=0
> > >> > > ).
> > >> > > > I'd propose we align
> > >> > > > KIP-345 along with this idea, and hence do not add the
> > >> > expansion_timeout
> > >> > > as
> > >> > > > part of the protocol layer, but only do that at the
> application's
> > >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> > still,
> > >> > > > deprecate the "*group.initial.rebalance.delay.ms
> > >> > > > <
> > >> > > >
> > >> > >
> > >> >
> > >>
> >
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=S8Ds6UNM56%2B3rq%2F%2BvYwKdMmzXrH1B5D3ghyjm06hu4g%3D&amp;reserved=0
> > >> > > >*"
> > >> > > > though as part of this KIP
> > >> > > > since we have discussed about its limit and think it is actually
> > >> not a
> > >> > > very
> > >> > > > good design and could be replaced with client-side logic above.
> > >> > > >
> > >> > > >
> > >> > > > 2. I'd like to see your thoughts on the upgrade path for this
> KIP.
> > >> More
> > >> > > > specifically, let's say after we have upgraded broker version to
> > be
> > >> > able
> > >> > > to
> > >> > > > recognize the new versions of JoinGroup request and the admin
> > >> requests,
> > >> > > how
> > >> > > > should we upgrade the clients and enable static groups? On top
> of
> > my
> > >> > head
> > >> > > > if we do a rolling bounce in which we set the member.name
> config
> > as
> > >> > well
> > >> > > > as
> > >> > > > optionally increase the session.timeout config when we bounce
> each
> > >> > > > instance, then during this rolling bounces we will have a group
> > >> > contained
> > >> > > > with both dynamic members and static members. It means that we
> > >> should
> > >> > > have
> > >> > > > the group to allow such scenario (i.e. we cannot reject
> JoinGroup
> > >> > > requests
> > >> > > > from dynamic members), and hence the "member.name" -> "
> member.id"
> > >> > > mapping
> > >> > > > will only be partial at this scenario. Also could you describe
> if
> > >> the
> > >> > > > upgrade to the first version that support this feature would
> ever
> > >> get
> > >> > any
> > >> > > > benefits, or only the future upgrade path for rolling bounces
> > could
> > >> get
> > >> > > > benefits out of this feature?
> > >> > > >
> > >> > > > If that's the case and we will do 1) as suggested above, do we
> > still
> > >> > need
> > >> > > > the enableStaticMembership and enableDynamicMembership admin
> > >> requests
> > >> > any
> > >> > > > more? Seems it is not necessary any more as we will only have
> the
> > >> > notion
> > >> > > of
> > >> > > > "dynamic or static members" that can co-exist in a group while
> > >> there no
> > >> > > > notion of "dynamic or static groups", and hence these two
> requests
> > >> are
> > >> > > not
> > >> > > > needed anymore.
> > >> > > >
> > >> > > >
> > >> > > > 3. We need to briefly talk about the implications for ACL as we
> > >> > introduce
> > >> > > > new admin requests that are related to a specific group.id. For
> > >> > example,
> > >> > > > we
> > >> > > > need to make sure that whoever created the group or joined the
> > group
> > >> > can
> > >> > > > actually send admin requests for the group, otherwise the
> > >> application
> > >> > > > owners need to bother the Kafka operators on a multi-tenant
> > cluster
> > >> > every
> > >> > > > time they want to send any admin requests for their groups which
> > >> would
> > >> > be
> > >> > > > an operational nightmare.
> > >> > > >
> > >> > > >
> > >> > > > 4. I like Jason's suggestion of adding an optional field for the
> > >> list
> > >> > of
> > >> > > > member names, and I'm wondering if that can be done as part of
> the
> > >> > > > forceStaticRebalance request: i.e. by passing a list of members,
> > we
> > >> > will
> > >> > > > enforce a rebalance immediately since it indicates that some
> > static
> > >> > > member
> > >> > > > will be officially kicked out of the group and some new static
> > >> members
> > >> > > may
> > >> > > > be added. So back to 1.a) above, a static member can only be
> > kicked
> > >> out
> > >> > > of
> > >> > > > the group if a) its session (arguably long period of time) has
> > timed
> > >> > out,
> > >> > > > and b) this admin request explicitly state that it is no longer
> > >> part of
> > >> > > the
> > >> > > > group. As for execution I'm fine with keeping it as a future
> work
> > of
> > >> > this
> > >> > > > KIP if you'd like to make its scope smaller.
> > >> > > >
> > >> > > > Following are minor comments:
> > >> > > >
> > >> > > > 5. I'm not sure if we need to include "member.name" as part of
> > the
> > >> > > > OffsetCommitRequest for fencing purposes, as I think the
> memberId
> > >> plus
> > >> > > the
> > >> > > > generation number should be sufficient for fencing even with
> > static
> > >> > > > members.
> > >> > > >
> > >> > > > 6. As mentioned above, if we agree to do 1) we can get rid of
> the
> > "
> > >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > >> > > >
> > >> > > >
> > >> > > > Guozhang
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > >
> > >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com>
> > >> wrote:
> > >> > > >
> > >> > > > > Hey Boyang,
> > >> > > > >
> > >> > > > > Thanks for the proposal! This is very useful. I have some
> > comments
> > >> > > below:
> > >> > > > >
> > >> > > > > 1) The motivation currently explicitly states that the goal is
> > to
> > >> > > improve
> > >> > > > > performance for heavy state application. It seems that the
> > >> motivation
> > >> > > can
> > >> > > > > be stronger with the following use-case. Currently for
> > MirrorMaker
> > >> > > > cluster
> > >> > > > > with e.g. 100 MirrorMaker processes, it will take a long time
> to
> > >> > > rolling
> > >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker
> process
> > >> > restart
> > >> > > > > will trigger a rebalance which currently pause the consumption
> > of
> > >> the
> > >> > > all
> > >> > > > > partitions of the MirrorMaker cluster. With the change stated
> in
> > >> this
> > >> > > > > patch, as long as a MirrorMaker can restart within the
> specified
> > >> > > timeout
> > >> > > > > (e.g. 2 minutes), then we only need constant number of
> rebalance
> > >> > (e.g.
> > >> > > > for
> > >> > > > > leader restart) for the entire rolling bounce, which will
> > >> > significantly
> > >> > > > > improves the availability of the MirrorMaker pipeline. In my
> > >> opinion,
> > >> > > the
> > >> > > > > main benefit of the KIP is to avoid unnecessary rebalance if
> the
> > >> > > consumer
> > >> > > > > process can be restarted within soon, which helps performance
> > >> even if
> > >> > > > > overhead of state shuffling for a given process is small.
> > >> > > > >
> > >> > > > > 2) In order to simplify the KIP reading, can you follow the
> > >> writeup
> > >> > > style
> > >> > > > > of other KIP (e.g. KIP-98) and list the interface change such
> as
> > >> new
> > >> > > > > configs (e.g. registration timeout), new request/response, new
> > >> > > > AdminClient
> > >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)?
> Currently
> > >> some
> > >> > > of
> > >> > > > > these are specified in the Proposed Change section which makes
> > it
> > >> a
> > >> > bit
> > >> > > > > inconvenient to understand the new interface that will be
> > exposed
> > >> to
> > >> > > > user.
> > >> > > > > Explanation of the current two-phase rebalance protocol
> probably
> > >> can
> > >> > be
> > >> > > > > moved out of public interface section.
> > >> > > > >
> > >> > > > > 3) There are currently two version of JoinGroupRequest in the
> > KIP
> > >> and
> > >> > > > only
> > >> > > > > one of them has field memberId. This seems confusing.
> > >> > > > >
> > >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> > >> rebalance
> > >> > > could
> > >> > > > > be helpful here, but we will make a call once we finished the
> > >> major
> > >> > > > > implementation". So this seems to be still an open question in
> > the
> > >> > > > current
> > >> > > > > design. We probably want to agree on this before voting for
> the
> > >> KIP.
> > >> > > > >
> > >> > > > > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> > Can
> > >> > you
> > >> > > > > specify the name of the config key and the default config
> value?
> > >> > > Possible
> > >> > > > > default values include empty string or null (similar to
> > >> > transaction.id
> > >> > > > in
> > >> > > > > producer config).
> > >> > > > >
> > >> > > > > 6) Regarding the use of the topic "static_member_map" to
> persist
> > >> > member
> > >> > > > > name map, currently if consumer coordinator broker goes
> offline,
> > >> > > > rebalance
> > >> > > > > is triggered and consumers will try connect to the new
> > >> coordinator.
> > >> > If
> > >> > > > > these consumers can connect to the new coordinator within
> > >> > > > > max.poll.interval.ms which by default is 5 minutes, given
> that
> > >> > broker
> > >> > > > can
> > >> > > > > use a deterministic algorithm to determine the partition ->
> > >> > member_name
> > >> > > > > mapping, each consumer should get assigned the same set of
> > >> partitions
> > >> > > > > without requiring state shuffling. So it is not clear whether
> we
> > >> > have a
> > >> > > > > strong use-case for this new logic. Can you help clarify what
> is
> > >> the
> > >> > > > > benefit of using topic "static_member_map" to persist member
> > name
> > >> > map?
> > >> > > > >
> > >> > > > > 7) Regarding the introduction of the expensionTimeoutMs
> config,
> > >> it is
> > >> > > > > mentioned that "we are using expansion timeout to replace
> > >> rebalance
> > >> > > > > timeout, which is configured by max.poll.intervals from client
> > >> side,
> > >> > > and
> > >> > > > > using registration timeout to replace session timeout".
> > Currently
> > >> the
> > >> > > > > default max.poll.interval.ms is configured to be 5 minutes
> and
> > >> there
> > >> > > > will
> > >> > > > > be only one rebalance if all new consumers can join within 5
> > >> minutes.
> > >> > > So
> > >> > > > it
> > >> > > > > is not clear whether we have a strong use-case for this new
> > >> config.
> > >> > Can
> > >> > > > you
> > >> > > > > explain what is the benefit of introducing this new config?
> > >> > > > >
> > >> > > > > 8) It is mentioned that "To distinguish between previous
> version
> > >> of
> > >> > > > > protocol, we will also increase the join group request version
> > to
> > >> v4
> > >> > > when
> > >> > > > > MEMBER_NAME is set" and "If the broker version is not the
> latest
> > >> (<
> > >> > > v4),
> > >> > > > > the join group request shall be downgraded to v3 without
> setting
> > >> the
> > >> > > > member
> > >> > > > > Id". It is probably simpler to just say that this feature is
> > >> enabled
> > >> > if
> > >> > > > > JoinGroupRequest V4 is supported on both client and broker and
> > >> > > > MEMBER_NAME
> > >> > > > > is configured with non-empty string.
> > >> > > > >
> > >> > > > > 9) It is mentioned that broker may return
> > >> NO_STATIC_MEMBER_INFO_SET
> > >> > > error
> > >> > > > > in OffsetCommitResponse for "commit requests under static
> > >> > membership".
> > >> > > > Can
> > >> > > > > you clarify how broker determines whether the commit request
> is
> > >> under
> > >> > > > > static membership?
> > >> > > > >
> > >> > > > > Thanks,
> > >> > > > > Dong
> > >> > > > >
> > >> > > >
> > >> > > >
> > >> > > > --
> > >> > > > -- Guozhang
> > >> > > >
> > >> > >
> > >> >
> > >> >
> > >> > --
> > >> > -Regards,
> > >> > Mayuresh R. Gharat
> > >> > (862) 250-7125
> > >> >
> > >>
> > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Mayuresh and Jason for your follow-ups! Let me try to answer both in this reply.


>    1. Do you intend to have member.id is a static config like member.name
>    after KIP-345 and KIP-394?

No, we shall only rely on broker to allocate member.id for the consumer instances. FYI, I already

started the discussion thread for KIP-394 😊

>    2. Regarding "On client side, we add a new config called MEMBER_NAME in
>    ConsumerConfig. On consumer service init, if the MEMBER_NAME config is
> set,
>    we will put it in the initial join group request to identify itself as a
>    static member (static membership); otherwise, we will still send
>    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID (dynamic
>    membership)."
>       - What is the value of member_id sent in the first JoinGroupRequest
>       when member_name is set (using static rebalance)? Is it
> UNKNOW_MEMBER_ID?

Yes, we could only use unknown member id. Actually this part of the proposal is outdated,

let me do another audit of the whole doc. Basically, it is currently impossible to send `member.id`

when consumer restarted. Sorry for the confusions!

>    3. Regarding "we are requiring member.id (if not unknown) to match the
>    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge case
>    that if we could have members with the same `member.name` (for example
>    mis-configured instances with a valid member.id but added a used member
>    name on runtime). When member name has duplicates, we could refuse join
>    request from members with an outdated `member.id` (since we update the
>    mapping upon each join group request). In an edge case where the client
>    hits this exception in the response, it is suggesting that some other
>    consumer takes its spot."
>       - The part of "some other consumer takes the spot" would be
>       intentional, right? Also when you say " The edge case that if we
>       could have members with the same `member.name` (for example
>       mis-configured instances *with a valid member.id <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0>
> *but
>       added a used member name on runtime).", what do you mean by *valid
>       member id* here? Does it mean that there exist a mapping of
>       member.name to member.id like *MemberA -> id1* on the
>       GroupCoordinator and this consumer is trying to join with *
> member.name
>       <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0> = MemberB and member.id <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0> =
> id1 *
>       ?

I would take Jason's advice that each time we have unknown member joining the group, the broker will

always assign a new and unique id to track its identity. In this way, consumer with duplicate member name

will be fenced.

>    4. Depending on your explanation for point 2 and the point 3 above
>    regarding returning back MEMBER_ID_MISMATCH on having a matching
>    member_name but unknown member_id, if the consumer sends
> "UNKNOW_MEMBER_ID"
>    on the first JoinGroupRequest and relies on the GroupCoordinator to
> give it
>    a member_id, is the consumer suppose to remember member_id for
>    joinGroupRequests? If yes, how are restarts handled?

Like explained above, we shall not materialize the member.id. Instead we need to rely on broker to allocate

a unique id for consumer just like what we have now.

>    5. Regarding "So in summary, *the member will only be removed due to
>    session timeout*. We shall remove it from both in-memory static member
>    name mapping and member list."
>       - If the rebalance is invoked manually using the the admin apis, how
>       long should the group coordinator wait for the members of the
> group to send
>       a JoinGroupRequest for participating in the rebalance? How is a
> lagging
>       consumer handled?

The plan is to disable member kick out when rebalance.timeout is reached, so basically we are not "waiting" any

join group request from existing members; we shall just rebalance base on what we currently have within the group

metadata. Lagging consumer will trigger rebalance later if session timeout > rebalance timeout.

>    6. Another detail to take care is that we need to automatically take the
>    hash of group id so that we know which broker to send this request to.
>       - I assume this should be same as the way we find the coordinator,
>       today right? If yes, should we specify it in the KIP ?

Yep, it is. Add FindCoordinatorRequest logic to the script.

>    7. Are there any specific failure scenarios when you say "other
>    potential failure cases."? It would be good to mention them explicitly,
> if
>    you think there are any.

Nah, I'm gonna remove it because it seems causing more confusion than making my assumption clear, which is

"there could be other failure cases that I can't enumerate now" 😊

>    8. It would be good to have a rollback plan as you have for roll forward
>    in the KIP.

Great suggestion! Added a simple rollback plan.


Next is answering Jason's suggestions:

1. This may be the same thing that Mayuresh is asking about. I think the
suggestion in the KIP is that if a consumer sends JoinGroup with a member
name, but no member id, then we will return the current member id
associated with that name. It seems in this case that we wouldn't be able
to protect from having two consumers active with the same configured
member.name? For example, imagine that we had a consumer with member.name=A
which is assigned member.id=1. Suppose it becomes a zombie and a new
instance starts up with member.name=A. If it is also assigned member.id=1,
then how can we detect the zombie if it comes back to life? Both instances
will have the same member.id.

The goal is to avoid a rebalance on a rolling restart, but we still need to
fence previous members. I am wondering if we can generate a new member.id
every time we receive a request from a static member with an unknown member
id. If the old instance with the same member.name attempts any operation,
then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
subscription of the new instance hasn't changed, then we can skip the
rebalance and return the current assignment without forcing a rebalance.

The trick to making this work is in the error handling of the zombie
consumer. If the zombie simply resets its member.id and rejoins to get a
new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
fencing the new member. We want to avoid this. There needs to be an
expectation for static members that the member.id of a static member will
not be changed except when a new member with the same member.name joins the
group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
with static member names.

Yep, I like this idea! Keep giving out refresh member.id when facing anonymous request will definitely

prevent processing bug due to duplicate consumers, however I don't think I fully understand the 3rd paragraph where

you mentioned  "There needs to be an expectation for static members that the member.id of a static member will

not be changed except when a new member with the same member.name joins the group. "  How do you plan
to know whether this member is new member or old member? I feel even with zombie consumer takes the ownership,
it should be detected very quickly (as MISMATCH_ID exception trigger original consumer instance dies)
and end user will start to fix it right away. Is there any similar logic we applied in fencing duplicate `transaction.id`?

2. The mechanics of the ConsumerRebalance API seem unclear to me. As far as
I understand it, it is used for scaling down a consumer group and somehow
bypasses normal session timeout expiration. I am wondering how critical
this piece is and whether we can leave it for future work. If not, then it
would be helpful to elaborate on its implementation. How would the
coordinator know which members to kick out of the group?

This API is needed when we need to immediately trigger rebalance instead of waiting session timeout

or rebalance timeout (Emergent scale up/down). It is very necessary to have it for

management purpose because user could choose when to trigger rebalance pretty freely,

gaining more client side control.

In the meanwhile I see your point that we need to actually have the ability to kick out members that we plan

to scale down fast (as rebalance timeout no longer kicks any offline member out of the group), I will think of adding an optional

list of members that are ready to be removed.

Another idea is to let static member send `LeaveGroupRequest` when they are going offline (either scale down or bouncing),

and broker will cache this information as "OfflineMembers" without triggering rebalance. When handling ConsumerRebalanceRequest broker will

kick the static members that are currently offline and trigger rebalance immediately. How does this plan sound?

3. I've been holding back on mentioning this, but I think we should
reconsider the name `member.name`. I think we want something that suggests
its expectation of uniqueness in the group. How about `group.instance.id`
to go along with `group.id`?

Yea, Dong and Stanislav also mentioned this naming. I personally buy in the namespace idea, and

since we already use `member.name` in a lot of context, I decide to rename the config to `group.member.name`

which should be sufficient for solving all the concerns we have now. Sounds good?


Thank you for your great suggestions! Let me know if my reply makes sense her.


Best,

Boyang

________________________________
From: Jason Gustafson <ja...@confluent.io>
Sent: Tuesday, November 27, 2018 7:51 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

Thanks for the updates. Looks like we're headed in the right direction and
clearly the interest that this KIP is receiving shows how strong the
motivation is!

I have a few questions:

1. This may be the same thing that Mayuresh is asking about. I think the
suggestion in the KIP is that if a consumer sends JoinGroup with a member
name, but no member id, then we will return the current member id
associated with that name. It seems in this case that we wouldn't be able
to protect from having two consumers active with the same configured
member.name? For example, imagine that we had a consumer with member.name=A
which is assigned member.id=1. Suppose it becomes a zombie and a new
instance starts up with member.name=A. If it is also assigned member.id=1,
then how can we detect the zombie if it comes back to life? Both instances
will have the same member.id.

The goal is to avoid a rebalance on a rolling restart, but we still need to
fence previous members. I am wondering if we can generate a new member.id
every time we receive a request from a static member with an unknown member
id. If the old instance with the same member.name attempts any operation,
then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
subscription of the new instance hasn't changed, then we can skip the
rebalance and return the current assignment without forcing a rebalance.

The trick to making this work is in the error handling of the zombie
consumer. If the zombie simply resets its member.id and rejoins to get a
new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
fencing the new member. We want to avoid this. There needs to be an
expectation for static members that the member.id of a static member will
not be changed except when a new member with the same member.name joins the
group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
with static member names.

2. The mechanics of the ConsumerRebalance API seem unclear to me. As far as
I understand it, it is used for scaling down a consumer group and somehow
bypasses normal session timeout expiration. I am wondering how critical
this piece is and whether we can leave it for future work. If not, then it
would be helpful to elaborate on its implementation. How would the
coordinator know which members to kick out of the group?

3. I've been holding back on mentioning this, but I think we should
reconsider the name `member.name`. I think we want something that suggests
its expectation of uniqueness in the group. How about `group.instance.id`
to go along with `group.id`?

Thanks,
Jason



On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> Hi Boyang,
>
> Thanks a lot for replying to all the queries and discussions here, so
> patiently.
> Really appreciate it.
>
> Had a few questions and suggestions after rereading the current version of
> the KIP :
>
>
>    1. Do you intend to have member.id is a static config like member.name
>    after KIP-345 and KIP-394?
>    2. Regarding "On client side, we add a new config called MEMBER_NAME in
>    ConsumerConfig. On consumer service init, if the MEMBER_NAME config is
> set,
>    we will put it in the initial join group request to identify itself as a
>    static member (static membership); otherwise, we will still send
>    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID (dynamic
>    membership)."
>       - What is the value of member_id sent in the first JoinGroupRequest
>       when member_name is set (using static rebalance)? Is it
> UNKNOW_MEMBER_ID?
>    3. Regarding "we are requiring member.id (if not unknown) to match the
>    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge case
>    that if we could have members with the same `member.name` (for example
>    mis-configured instances with a valid member.id but added a used member
>    name on runtime). When member name has duplicates, we could refuse join
>    request from members with an outdated `member.id` (since we update the
>    mapping upon each join group request). In an edge case where the client
>    hits this exception in the response, it is suggesting that some other
>    consumer takes its spot."
>       - The part of "some other consumer takes the spot" would be
>       intentional, right? Also when you say " The edge case that if we
>       could have members with the same `member.name` (for example
>       mis-configured instances *with a valid member.id <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0>
> *but
>       added a used member name on runtime).", what do you mean by *valid
>       member id* here? Does it mean that there exist a mapping of
>       member.name to member.id like *MemberA -> id1* on the
>       GroupCoordinator and this consumer is trying to join with *
> member.name
>       <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0> = MemberB and member.id <https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.id&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=cGI1RvreeJNeHgWkXu6ZZ%2BIMzD8uR2y9OKaQKc7Vsf8%3D&amp;reserved=0> =
> id1 *
>       ?
>    4. Depending on your explanation for point 2 and the point 3 above
>    regarding returning back MEMBER_ID_MISMATCH on having a matching
>    member_name but unknown member_id, if the consumer sends
> "UNKNOW_MEMBER_ID"
>    on the first JoinGroupRequest and relies on the GroupCoordinator to
> give it
>    a member_id, is the consumer suppose to remember member_id for
>    joinGroupRequests? If yes, how are restarts handled?
>    5. Regarding "So in summary, *the member will only be removed due to
>    session timeout*. We shall remove it from both in-memory static member
>    name mapping and member list."
>       - If the rebalance is invoked manually using the the admin apis, how
>       long should the group coordinator wait for the members of the
> group to send
>       a JoinGroupRequest for participating in the rebalance? How is a
> lagging
>       consumer handled?
>    6. Another detail to take care is that we need to automatically take the
>    hash of group id so that we know which broker to send this request to.
>       - I assume this should be same as the way we find the coordinator,
>       today right? If yes, should we specify it in the KIP ?
>    7. Are there any specific failure scenarios when you say "other
>    potential failure cases."? It would be good to mention them explicitly,
> if
>    you think there are any.
>    8. It would be good to have a rollback plan as you have for roll forward
>    in the KIP.
>
> Thanks,
>
> Mayuresh
>
> On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Do you have a discuss thread for KIP-394 that you mentioned here ?
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> >> Hey Dong, thanks for the follow-up here!
> >>
> >>
> >> 1) It is not very clear to the user what is the difference between
> >> member.name and client.id as both seems to be used to identify the
> >> consumer. I am wondering if it would be more intuitive to name it
> >> group.member.name (preferred choice since it matches the current
> group.id
> >> config name) or rebalance.member.name to explicitly show that the id is
> >> solely used for rebalance.
> >> Great question. I feel `member.name` is enough to explain itself, it
> >> seems not very
> >> helpful to make the config name longer. Comparing `name` with `id` gives
> >> user the
> >> impression that they have the control over it with customized rule than
> >> library decided.
> >>
> >> 2) In the interface change section it is said that
> >> GroupMaxSessionTimeoutMs
> >> will be changed to 30 minutes. It seems to suggest that we will change
> the
> >> default value of this config. It does not seem necessary to increase the
> >> time of consumer failure detection when user doesn't use static
> >> membership.
> >> Also, say static membership is enabled, then this default config change
> >> will cause a partition to be unavailable for consumption for 30 minutes
> if
> >> there is hard consumer failure, which seems to be worse experience than
> >> having unnecessary rebalance (when this timeout is small), particularly
> >> for
> >> new users of Kafka. Could you explain more why we should make this
> change?
> >> We are not changing the default session timeout value. We are just
> >> changing the
> >> cap we are enforcing on the session timeout max value. So this change is
> >> not affecting
> >> what kind of membership end user is using, and loosing the cap is giving
> >> end user
> >> more flexibility on trade-off between liveness and stability.
> >>
> >> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> >> into one error? It seems that these two errors are currently handled by
> >> the
> >> consumer in the same way. And we don't also don't expect
> >> MEMBER_ID_MISMATCH
> >> to happen. Thus it is not clear what is the benefit of having two
> errors.
> >> I agree that we should remove DUPLICATE_STATIC_MEMBER error because with
> >> the KIP-394<
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-394%253A%2BRequire%2Bmember%2Bid%2Bfor%2Binitial%2Bjoin%2Bgroup%2Brequest&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=EDM7PmpOo2HenYhFHX2rxrszpkI7di401WhKh2Vjw5k%3D&amp;reserved=0
> >> >
> >> we will automatically fence all join requests with UNKNOWN_MEMBER_ID.
> >>
> >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> contains
> >> member name which is already in the consumer group, however the member
> id
> >> was missing". After a consumer is restarted, it will send a
> >> JoinGroupRequest with an existing memberName (as the coordinator has not
> >> expired this member from the memory) and memberId
> >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> >> across consumer restart in the consumer side). Does it mean that
> >> JoinGroupRequest from a newly restarted consumer will always be rejected
> >> until the sessionTimeoutMs has passed?
> >> Same answer as question 3). This part of the logic shall be removed from
> >> the proposal.
> >>
> >> 5) It seems that we always add two methods to the interface
> >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> the
> >> other without option. Could this be specified in the interface change
> >> section?
> >> Sounds good! Added both methods.
> >>
> >> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> >> rebalance? If so, we probably want to specify the command line tool
> >> interface similar to
> >>
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> >> .
> >> Added the script.
> >>
> >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> >> "invokeConsumerRebalance"? It is not very clear what is the extra
> meaning
> >> of world "force" as compared to "trigger" or "invoke". And it seems
> >> simpler
> >> to allows this API to trigger rebalance regardless of whether consumer
> is
> >> configured with memberName.
> >> Sounds good. Right now I feel for both static and dynamic membership it
> is
> >> more manageable to introduce the consumer rebalance method through admin
> >> client API.
> >>
> >> 8) It is not very clear how the newly added AdminClient API trigger
> >> rebalance. For example, does it send request? Can this be explained in
> the
> >> KIP?
> >>
> >> Sure, I will add more details to the API.
> >>
> >>
> >> Thanks again for the helpful suggestions!
> >>
> >>
> >> Best,
> >> Boyang
> >>
> >> ________________________________
> >> From: Dong Lin <li...@gmail.com>
> >> Sent: Saturday, November 24, 2018 2:54 PM
> >> To: dev
> >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> specifying member id
> >>
> >> Hey Boyang,
> >>
> >> Thanks for the update! Here are some followup comments:
> >>
> >> 1) It is not very clear to the user what is the difference between
> >> member.name and client.id as both seems to be used to identify the
> >> consumer. I am wondering if it would be more intuitive to name it
> >> group.member.name (preferred choice since it matches the current
> group.id
> >> config name) or rebalance.member.name to explicitly show that the id is
> >> solely used for rebalance.
> >>
> >> 2) In the interface change section it is said that
> >> GroupMaxSessionTimeoutMs
> >> will be changed to 30 minutes. It seems to suggest that we will change
> the
> >> default value of this config. It does not seem necessary to increase the
> >> time of consumer failure detection when user doesn't use static
> >> membership.
> >> Also, say static membership is enabled, then this default config change
> >> will cause a partition to be unavailable for consumption for 30 minutes
> if
> >> there is hard consumer failure, which seems to be worse experience than
> >> having unnecessary rebalance (when this timeout is small), particularly
> >> for
> >> new users of Kafka. Could you explain more why we should make this
> change?
> >>
> >> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> >> into one error? It seems that these two errors are currently handled by
> >> the
> >> consumer in the same way. And we don't also don't expect
> >> MEMBER_ID_MISMATCH
> >> to happen. Thus it is not clear what is the benefit of having two
> errors.
> >>
> >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> contains
> >> member name which is already in the consumer group, however the member
> id
> >> was missing". After a consumer is restarted, it will send a
> >> JoinGroupRequest with an existing memberName (as the coordinator has not
> >> expired this member from the memory) and memberId
> >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> >> across consumer restart in the consumer side). Does it mean that
> >> JoinGroupRequest from a newly restarted consumer will always be rejected
> >> until the sessionTimeoutMs has passed?
> >>
> >> 5) It seems that we always add two methods to the interface
> >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> the
> >> other without option. Could this be specified in the interface change
> >> section?
> >>
> >> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> >> rebalance? If so, we probably want to specify the command line tool
> >> interface similar to
> >>
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=I1zGk61VzDvoGfPfzSbOms4l9g%2BrXQvttKsfNNwuuJ4%3D&amp;reserved=0
> >> .
> >>
> >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> >> "invokeConsumerRebalance"? It is not very clear what is the extra
> meaning
> >> of world "force" as compared to "trigger" or "invoke". And it seems
> >> simpler
> >> to allows this API to trigger rebalance regardless of whether consumer
> is
> >> configured with memberName.
> >>
> >> 8) It is not very clear how the newly added AdminClient API trigger
> >> rebalance. For example, does it send request? Can this be explained in
> the
> >> KIP?
> >>
> >> Thanks,
> >> Dong
> >>
> >>
> >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> wrote:
> >>
> >> > Hey Mayuresh,
> >> >
> >> >
> >> > thanks for your feedbacks! I will try do another checklist here.
> >> >
> >> >
> >> > > By this you mean, even if the application has not called
> >> > > KafkaConsumer.poll() within session timeout, it will not be sending
> >> the
> >> > > LeaveGroup request, right?
> >> >
> >> > Yep it's true, we will prevent client from sending leave group request
> >> > when they are set with `member.name`.
> >> >
> >> >
> >> > > When is the member.name removed from this map?
> >> > Good question, we will only kick off member due to session timeout
> >> within
> >> > static membership. Let me update the KIP to clearly assert that.
> >> >
> >> > > How is this case (missing member id) handled on the client side?
> What
> >> is
> >> > the application that
> >> > > is using the KafkaConsumer suppose to do in this scenario?
> >> > I have extended the two exceptions within join group response V4.
> >> > Basically I define both corresponding actions to be immediate failing
> >> > client application, because so far it is unknown what kind of client
> >> issue
> >> > could trigger them. After the first version, we will keep enhance the
> >> error
> >> > handling logic!
> >> >
> >> > > This would mean that it might take more time to detect unowned topic
> >> > > partitions and may cause delay for applications that perform data
> >> > mirroring
> >> > > tasks. I discussed this with our sre and we have a suggestion to
> make
> >> > here
> >> > > as listed below separately.
> >> > The goal of extending session timeout cap is for users with good
> client
> >> > side monitoring tools that could auto-heal the dead consumers very
> >> fast. So
> >> > it is optional (and personal) to extend session timeout to a
> reasonable
> >> > number with different client scenarios.
> >> >
> >> > > you meant remove unjoined members of the group, right ?
> >> > Yep, there is a typo. Thanks for catching this!
> >> >
> >> > > What do you mean by " Internally we would optimize this logic by
> >> having
> >> > > rebalance timeout only in charge of stopping prepare rebalance
> stage,
> >> > > without removing non-responsive members immediately." There would
> not
> >> be
> >> > a
> >> > > full rebalance if the lagging consumer sent a JoinGroup request
> later,
> >> > > right ? If yes, can you highlight this in the KIP ?
> >> > No, there won't be. We want to limit the rebalance timeout
> functionality
> >> > to only use as a timer to
> >> > end prepare rebalance stage. This way, late joining static members
> will
> >> > not trigger further rebalance
> >> > as long as they are within session timeout. I added your highlight to
> >> the
> >> > KIP!
> >> >
> >> > > The KIP talks about scale up scenario but its not quite clear how we
> >> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> >> > status
> >> > > "learner" ?. Can you shed more light on how this is handled in the
> >> KIP,
> >> > if
> >> > > its handled?
> >> > Updated the KIP: we shall not cover scale up case in 345, because we
> >> > believe client side could
> >> > better handle this logic.
> >> >
> >> > > I think Jason had brought this up earlier about having a way to say
> >> how
> >> > > many members/consumer hosts are you choosing to be in the consumer
> >> group.
> >> > > If we can do this, then in case of mirroring applications we can do
> >> this
> >> > :
> >> > > Lets say we have a mirroring application that consumes from Kafka
> >> cluster
> >> > > A and produces to Kafka cluster B.
> >> > > Depending on the data and the Kafka cluster configuration, Kafka
> >> service
> >> > > providers can set a mirroring group saying that it will take, for
> >> example
> >> > > 300 consumer hosts/members to achieve the desired throughput and
> >> latency
> >> > > for mirroring and can have additional 10 consumer hosts as spare in
> >> the
> >> > > same group.
> >> > > So when the first 300 members/consumers to join the group will start
> >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> >> > > The remaining 10 consumer members can sit idle.
> >> > > The moment one of the consumer (for example: consumer number 54)
> from
> >> the
> >> > > first 300 members go out of the group (crossed session timeout), it
> >> (the
> >> > > groupCoordinator) can just assign the topicPartitions from the
> >> consumer
> >> > > member 54 to one of the spare hosts.
> >> > > Once the consumer member 54 comes back up, it can start as being a
> >> part
> >> > of
> >> > > the spare pool.
> >> > > This enables us to have lower session timeouts and low latency
> >> mirroring,
> >> > > in cases where the service providers are OK with having spare hosts.
> >> > > This would mean that we would tolerate n consumer members leaving
> and
> >> > > rejoining the group and still provide low latency as long as n <=
> >> number
> >> > of
> >> > > spare consumers.
> >> > > If there are no spare host available, we can get back to the idea as
> >> > > described in the KIP.
> >> > Great idea! In fact on top of static membership we could later
> introduce
> >> > APIs to set hard-coded
> >> > client ids to the group and replace the dead host, or as you proposed
> to
> >> > define spare host as
> >> > what I understood as hot backup. I will put both Jason and your
> >> > suggestions into a separate section
> >> > called "Future works". Note that this spare host idea may be also
> >> solvable
> >> > through rebalance protocol
> >> > IMO.
> >> >
> >> > Thank you again for the great feedback!
> >> >
> >> > Boyang
> >> > ________________________________
> >> > From: Boyang Chen <bc...@outlook.com>
> >> > Sent: Thursday, November 22, 2018 3:39 PM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> > specifying member id
> >> >
> >> > Hey Dong, sorry for missing your message. I couldn't find your email
> on
> >> my
> >> > thread, so I will just do a checklist here!
> >> >
> >> >
> >> > 1) The motivation currently explicitly states that the goal is to
> >> improve
> >> >
> >> > performance for heavy state application. It seems that the motivation
> >> can
> >> >
> >> > be stronger with the following use-case. Currently for MirrorMaker
> >> cluster
> >> >
> >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> rolling
> >> >
> >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> restart
> >> >
> >> > will trigger a rebalance which currently pause the consumption of the
> >> all
> >> >
> >> > partitions of the MirrorMaker cluster. With the change stated in this
> >> >
> >> > patch, as long as a MirrorMaker can restart within the specified
> timeout
> >> >
> >> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> >> for
> >> >
> >> > leader restart) for the entire rolling bounce, which will
> significantly
> >> >
> >> > improves the availability of the MirrorMaker pipeline. In my opinion,
> >> the
> >> >
> >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> >> consumer
> >> >
> >> > process can be restarted within soon, which helps performance even if
> >> >
> >> > overhead of state shuffling for a given process is small.
> >> >
> >> > I just rephrased this part and added it to the KIP. Thanks for making
> >> the
> >> > motivation more solid!
> >> >
> >> > 2) In order to simplify the KIP reading, can you follow the writeup
> >> style
> >> > of other KIP (e.g. KIP-98) and list the interface change such as new
> >> > configs (e.g. registration timeout), new request/response, new
> >> AdminClient
> >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> of
> >> > these are specified in the Proposed Change section which makes it a
> bit
> >> > inconvenient to understand the new interface that will be exposed to
> >> user.
> >> > Explanation of the current two-phase rebalance protocol probably can
> be
> >> > moved out of public interface section.
> >> > This is a great suggestion! I just consolidated all the public API
> >> > changes, and the whole KIP
> >> > looks much more organized!
> >> >
> >> > 3) There are currently two version of JoinGroupRequest in the KIP and
> >> only
> >> > one of them has field memberId. This seems confusing.
> >> > Yep, I already found this issue and fixed it.
> >> >
> >> > 4) It is mentioned in the KIP that "An admin API to force rebalance
> >> could
> >> > be helpful here, but we will make a call once we finished the major
> >> > implementation". So this seems to be still an open question in the
> >> current
> >> > design. We probably want to agree on this before voting for the KIP.
> >> > We have finalized the idea that this API is needed.
> >> >
> >> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> >> > specify the name of the config key and the default config value?
> >> Possible
> >> > default values include empty string or null (similar to
> transaction.id<
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=N8RxV6%2Bh7ib9CMpW3ZyFq3m2awY1sRPHzlOTi6qU5XY%3D&amp;reserved=0
> >> >
> >> > in
> >> > producer config).
> >> > I have defined the `member.name` in "New configuration" section.
> >> >
> >> > 6) Regarding the use of the topic "static_member_map" to persist
> member
> >> > name map, currently if consumer coordinator broker goes offline,
> >> rebalance
> >> > is triggered and consumers will try connect to the new coordinator. If
> >> > these consumers can connect to the new coordinator within
> >> > max.poll.interval.ms<
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> >> >
> >> > which by default is 5 minutes, given that broker can
> >> > use a deterministic algorithm to determine the partition ->
> member_name
> >> > mapping, each consumer should get assigned the same set of partitions
> >> > without requiring state shuffling. So it is not clear whether we have
> a
> >> > strong use-case for this new logic. Can you help clarify what is the
> >> > benefit of using topic "static_member_map" to persist member name map?
> >> > I have discussed with Guozhang offline, and I believe reusing the
> >> current
> >> > `_consumer_offsets`
> >> > topic is a better and unified solution.
> >> >
> >> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> >> > mentioned that "we are using expansion timeout to replace rebalance
> >> > timeout, which is configured by max.poll.intervals from client side,
> and
> >> > using registration timeout to replace session timeout". Currently the
> >> > default max.poll.interval.ms<
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=F11nFuanrUtwwp1YVwvmYDsuV0mIs6QKt%2Bf2gPxD2t8%3D&amp;reserved=0
> >> >
> >> > is configured to be 5 minutes and there will
> >> > be only one rebalance if all new consumers can join within 5 minutes.
> >> So it
> >> > is not clear whether we have a strong use-case for this new config.
> Can
> >> you
> >> > explain what is the benefit of introducing this new config?
> >> > Previously our goal is to use expansion timeout as a workaround for
> >> > triggering multiple
> >> > rebalances when scaling up members are not joining at the same time.
> It
> >> is
> >> > decided to
> >> > be addressed by client side protocol change, so we will not introduce
> >> > expansion timeout.
> >> >
> >> > 8) It is mentioned that "To distinguish between previous version of
> >> > protocol, we will also increase the join group request version to v4
> >> when
> >> > MEMBER_NAME is set" and "If the broker version is not the latest (<
> v4),
> >> > the join group request shall be downgraded to v3 without setting the
> >> member
> >> > Id". It is probably simpler to just say that this feature is enabled
> if
> >> > JoinGroupRequest V4 is supported on both client and broker and
> >> MEMBER_NAME
> >> > is configured with non-empty string.
> >> > Yep, addressed this!
> >> >
> >> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> >> error
> >> > in OffsetCommitResponse for "commit requests under static membership".
> >> Can
> >> > you clarify how broker determines whether the commit request is under
> >> > static membership?
> >> >
> >> > We have agreed that commit request shouldn't be affected by the new
> >> > membership, thus
> >> > removing it here. Thanks for catching this!
> >> >
> >> > Let me know if you have further suggestions or concerns. Thank you for
> >> > your valuable feedback
> >> > to help me design the KIP better! (And I will try to address your
> >> > feedbacks in next round Mayuresh ??)
> >> >
> >> > Best,
> >> > Boyang
> >> > ________________________________
> >> > From: Mayuresh Gharat <gh...@gmail.com>
> >> > Sent: Wednesday, November 21, 2018 7:50 AM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> > specifying member id
> >> >
> >> > Hi Boyang,
> >> >
> >> > Thanks for updating the KIP. This is a step good direction for
> stateful
> >> > applications and also mirroring applications whose latency is affected
> >> due
> >> > to the rebalance issues that we have today.
> >> >
> >> > I had a few questions on the current version of the KIP :
> >> > For the effectiveness of the KIP, consumer with member.name set will
> >> *not
> >> > send leave group request* when they go offline
> >> >
> >> > > By this you mean, even if the application has not called
> >> > > KafkaConsumer.poll() within session timeout, it will not be sending
> >> the
> >> > > LeaveGroup request, right?
> >> > >
> >> >
> >> > Broker will maintain an in-memory mapping of {member.name ? member.id
> }
> >> to
> >> > track member uniqueness.
> >> >
> >> > > When is the member.name removed from this map?
> >> > >
> >> >
> >> > Member.id must be set if the *member.name <
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=zSs67QfaXFLcI%2FiWTB3nfHuQLCHeZZUdz5hvD%2Brh5ug%3D&amp;reserved=0
> >> >
> >> > *is already
> >> > within the map. Otherwise reply MISSING_MEMBER_ID
> >> >
> >> > > How is this case handled on the client side? What is the application
> >> that
> >> > > is using the KafkaConsumer suppose to do in this scenario?
> >> > >
> >> >
> >> > Session timeout is the timeout we will trigger rebalance when a member
> >> goes
> >> > offline for too long (not sending heartbeat request). To make static
> >> > membership effective, we should increase the default max session
> >> timeout to
> >> > 30 min so that end user could config it freely.
> >> >
> >> > > This would mean that it might take more time to detect unowned topic
> >> > > partitions and may cause delay for applications that perform data
> >> > mirroring
> >> > > tasks. I discussed this with our sre and we have a suggestion to
> make
> >> > here
> >> > > as listed below separately.
> >> > >
> >> >
> >> > Currently there is a config called *rebalance timeout* which is
> >> configured
> >> > by consumer *max.poll.intervals*. The reason we set it to poll
> interval
> >> is
> >> > because consumer could only send request within the call of poll() and
> >> we
> >> > want to wait sufficient time for the join group request. When reaching
> >> > rebalance timeout, the group will move towards completingRebalance
> stage
> >> > and remove unjoined groups
> >> >
> >> > > you meant remove unjoined members of the group, right ?
> >> > >
> >> >
> >> > Currently there is a config called *rebalance timeout* which is
> >> configured
> >> > by consumer *max.poll.intervals*. The reason we set it to poll
> interval
> >> is
> >> > because consumer could only send request within the call of poll() and
> >> we
> >> > want to wait sufficient time for the join group request. When reaching
> >> > rebalance timeout, the group will move towards completingRebalance
> stage
> >> > and remove unjoined groups. This is actually conflicting with the
> >> design of
> >> > static membership, because those temporarily unavailable members will
> >> > potentially reattempt the join group and trigger extra rebalances.
> >> > Internally we would optimize this logic by having rebalance timeout
> >> only in
> >> > charge of stopping prepare rebalance stage, without removing
> >> non-responsive
> >> > members immediately.
> >> >
> >> > > What do you mean by " Internally we would optimize this logic by
> >> having
> >> > > rebalance timeout only in charge of stopping prepare rebalance
> stage,
> >> > > without removing non-responsive members immediately." There would
> not
> >> be
> >> > a
> >> > > full rebalance if the lagging consumer sent a JoinGroup request
> later,
> >> > > right ? If yes, can you highlight this in the KIP ?
> >> > >
> >> >
> >> > Scale Up
> >> >
> >> > > The KIP talks about scale up scenario but its not quite clear how we
> >> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> >> > status
> >> > > "learner" ?. Can you shed more light on how this is handled in the
> >> KIP,
> >> > if
> >> > > its handled?
> >> > >
> >> >
> >> >
> >> > *Discussion*
> >> > Larger session timeouts causing latency rise for getting data for
> >> un-owned
> >> > topic partitions :
> >> >
> >> > > I think Jason had brought this up earlier about having a way to say
> >> how
> >> > > many members/consumer hosts are you choosing to be in the consumer
> >> group.
> >> > > If we can do this, then in case of mirroring applications we can do
> >> this
> >> > :
> >> > > Lets say we have a mirroring application that consumes from Kafka
> >> cluster
> >> > > A and produces to Kafka cluster B.
> >> > > Depending on the data and the Kafka cluster configuration, Kafka
> >> service
> >> > > providers can set a mirroring group saying that it will take, for
> >> example
> >> > > 300 consumer hosts/members to achieve the desired throughput and
> >> latency
> >> > > for mirroring and can have additional 10 consumer hosts as spare in
> >> the
> >> > > same group.
> >> > > So when the first 300 members/consumers to join the group will start
> >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> >> > > The remaining 10 consumer members can sit idle.
> >> > > The moment one of the consumer (for example: consumer number 54)
> from
> >> the
> >> > > first 300 members go out of the group (crossed session timeout), it
> >> (the
> >> > > groupCoordinator) can just assign the topicPartitions from the
> >> consumer
> >> > > member 54 to one of the spare hosts.
> >> > > Once the consumer member 54 comes back up, it can start as being a
> >> part
> >> > of
> >> > > the spare pool.
> >> > > This enables us to have lower session timeouts and low latency
> >> mirroring,
> >> > > in cases where the service providers are OK with having spare hosts.
> >> > > This would mean that we would tolerate n consumer members leaving
> and
> >> > > rejoining the group and still provide low latency as long as n <=
> >> number
> >> > of
> >> > > spare consumers.
> >> > > If there are no spare host available, we can get back to the idea as
> >> > > described in the KIP.
> >> > >
> >> >
> >> > Thanks,
> >> >
> >> > Mayuresh
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> >> > konstantine@confluent.io> wrote:
> >> >
> >> > > Hi Boyang.
> >> > >
> >> > > Thanks for preparing this KIP! It is making good progress and will
> be
> >> a
> >> > > great improvement for stateful Kafka applications.
> >> > >
> >> > > Apologies for my late reply, I was away for a while. Lots of great
> >> > comments
> >> > > so far, so I'll probably second most of them in what I suggest below
> >> at
> >> > > this point.
> >> > >
> >> > > When I first read the KIP, I wanted to start at the end with
> something
> >> > that
> >> > > wasn't highlighted a lot. That was the topic related to handling
> >> > duplicate
> >> > > members. I see now that the initial suggestion of handling this
> >> situation
> >> > > during offset commit has been removed, and I agree with that. Issues
> >> > > related to membership seem to be handled better when the member
> joins
> >> the
> >> > > group rather than when it tries to commit offsets. This also
> >> simplifies
> >> > how
> >> > > many request types need to change in order to incorporate the new
> >> member
> >> > > name field.
> >> > >
> >> > > I also agree with what Jason and Guozhang have said regarding
> >> timeouts.
> >> > > Although semantically, it's easier to think of every operation
> having
> >> its
> >> > > own timeout, operationally this can become a burden. Thus,
> >> consolidation
> >> > > seems preferable here. The definition of embedded protocols on top
> of
> >> the
> >> > > base group membership protocol for rebalancing gives enough
> >> flexibility
> >> > to
> >> > > address such needs in each client component separately.
> >> > >
> >> > > Finally, some minor comments:
> >> > > In a few places the new/proposed changes are referred to as
> "current".
> >> > > Which is a bit confusing considering that there is a protocol in
> place
> >> > > already, and by "current" someone might understand the existing one.
> >> I'd
> >> > > recommend using new/proposed or equivalent when referring to changes
> >> > > introduced with KIP-345 and current/existing or equivalent when
> >> referring
> >> > > to existing behavior.
> >> > >
> >> > > There's the following sentence in the "Public Interfaces" section:
> >> > > "Since for many stateful consumer/stream applications, the state
> >> > shuffling
> >> > > is more painful than short time partial unavailability."
> >> > > However, my understanding is that the changes proposed with KIP-345
> >> will
> >> > > not exploit any partial availability. A suggestion for dealing with
> >> > > temporary imbalances has been made in "Incremental Cooperative
> >> > Rebalancing"
> >> > > which can work well with KIP-345, but here I don't see proposed
> >> changes
> >> > > that suggest that some resources (e.g. partitions) will keep being
> >> used
> >> > > while others will not be utilized. Thus, you might want to adjust
> this
> >> > > sentence. Correct me if I'm missing something related to that.
> >> > >
> >> > > In the rejected alternatives, under point 2) I read "we can copy the
> >> > member
> >> > > id to the config files". I believe it means to say "member name"
> >> unless
> >> > I'm
> >> > > missing something about reusing member ids. Also below I read: "By
> >> > allowing
> >> > > consumers to optionally specifying a member id" which probably
> implies
> >> > > "member name" again. In a sense this section highlights a potential
> >> > > confusion between member name and member id. I wonder if we could
> >> come up
> >> > > with a better term for the new field. StaticTag, StaticLabel, or
> even
> >> > > StaticName are some suggestions that could potentially help with
> >> > confusion
> >> > > between MemberId and MemberName and what corresponds to what. But I
> >> > > wouldn't like to disrupt the discussion with naming conventions too
> >> much
> >> > at
> >> > > this point. I just mention it here as a thought.
> >> > >
> >> > > Looking forward to see the final details of this KIP. Great work so
> >> far!
> >> > >
> >> > > Konstantine
> >> > >
> >> > >
> >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com>
> >> wrote:
> >> > >
> >> > > > Thanks Guozhang for the great summary here, and I have been
> >> following
> >> > up
> >> > > > the action items here.
> >> > > >
> >> > > >
> >> > > >   1.  I already updated the KIP to remove the expansion timeout
> and
> >> > > > registration timeout. Great to see them being addressed in client
> >> side!
> >> > > >   2.  I double checked the design and I believe that it is ok to
> >> have
> >> > > both
> >> > > > static member and dynamic member co-exist in the same group. So
> the
> >> > > upgrade
> >> > > > shouldn't be destructive and we are removing the two membership
> >> > protocol
> >> > > > switching APIs.
> >> > > >   3.  I only have question about this one. I'm still reading the
> >> > > KafkaApis
> >> > > > code here. Should I just use the same authorization logic for
> >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> >> > > >   4.  I'm very excited to see this work with K8! Like you
> suggested,
> >> > this
> >> > > > feature could be better addressed in a separate KIP because it is
> >> > pretty
> >> > > > independent. I could start drafting the KIP once the current
> >> proposal
> >> > is
> >> > > > approved.
> >> > > >   5.  I believe that we don't need fencing in offset commit
> request,
> >> > > since
> >> > > > duplicate member.name issue could be handled by join group
> >> request. We
> >> > > > shall reject join group with known member name but no member id
> >> (which
> >> > > > means we already have an active member using this identity).
> >> > > >   6.  I agree to remove that internal config once we move forward
> >> with
> >> > > > static membership. And I already removed the entire section from
> the
> >> > KIP.
> >> > > >
> >> > > > Let me know if you have other concerns.
> >> > > >
> >> > > > Best,
> >> > > > Boyang
> >> > > > ________________________________
> >> > > > From: Guozhang Wang <wa...@gmail.com>
> >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> >> > > > To: dev
> >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> >> by
> >> > > > specifying member id
> >> > > >
> >> > > > Hello Boyang,
> >> > > >
> >> > > > Thanks a lot for the KIP! It is a great write-up and I appreciate
> >> your
> >> > > > patience answering to the feedbacks from the community. I'd like
> to
> >> add
> >> > > my
> >> > > > 2cents here:
> >> > > >
> >> > > > 1. By introducing another two timeout configs,
> registration_timeout
> >> and
> >> > > > expansion_timeout, we are effectively having four timeout configs:
> >> > > session
> >> > > > timeout, rebalance timeout (configured as "max.poll.interval.ms"
> on
> >> > > client
> >> > > > side), and these two. Interplaying these timeout configs can be
> >> quite
> >> > > hard
> >> > > > for users with such complexity, and hence I'm wondering if we can
> >> > > simplify
> >> > > > the situation with as less possible timeout configs as possible.
> >> Here
> >> > is
> >> > > a
> >> > > > concrete suggestion I'd like propose:
> >> > > >
> >> > > > 1.a) Instead of introducing a registration_timeout in addition to
> >> the
> >> > > > session_timeout for static members, we can just reuse the
> >> > session_timeout
> >> > > > and ask users to set it to a larger value when they are upgrading
> a
> >> > > dynamic
> >> > > > client to a static client by setting the "member.name" at the
> same
> >> > time.
> >> > > > By
> >> > > > default, the broker-side min.session.timeout is 6 seconds and
> >> > > > max.session.timeout is 5 minutes, which seems reasonable to me (we
> >> can
> >> > of
> >> > > > course modify this broker config to enlarge the valid interval if
> we
> >> > want
> >> > > > in practice). And then we should also consider removing the
> >> condition
> >> > for
> >> > > > marking a client as failed if the rebalance timeout has reached
> >> while
> >> > the
> >> > > > JoinGroup was not received, so that the semantics of
> session_timeout
> >> > and
> >> > > > rebalance_timeout are totally separated: the former is only used
> to
> >> > > > determine if a consumer member of the group should be marked as
> >> failed
> >> > > and
> >> > > > kicked out of the group, and the latter is only used to determine
> >> the
> >> > > > longest time coordinator should wait for PREPARE_REBALANCE phase.
> In
> >> > > other
> >> > > > words if a member did not send the JoinGroup in time of the
> >> > > > rebalance_timeout, we still include it in the new generation of
> the
> >> > group
> >> > > > and use its old subscription info to send to leader for
> assignment.
> >> > Later
> >> > > > if the member came back with HeartBeat request, we can still
> follow
> >> the
> >> > > > normal path to bring it to the latest generation while checking
> that
> >> > its
> >> > > > sent JoinGroup request contains the same subscription info as we
> >> used
> >> > to
> >> > > > assign the partitions previously (which should be likely the case
> in
> >> > > > practice). In addition, we should let static members to not send
> the
> >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> static
> >> > > member
> >> > > > can only be leaving the group if its session has timed out, OR it
> >> has
> >> > > been
> >> > > > indicated to not exist in the group any more (details below).
> >> > > >
> >> > > > 1.b) We have a parallel discussion about Incremental Cooperative
> >> > > > Rebalancing, in which we will encode the "when to rebalance" logic
> >> at
> >> > the
> >> > > > application level, instead of at the protocol level. By doing this
> >> we
> >> > can
> >> > > > also enable a few other optimizations, e.g. at the Streams level
> to
> >> > first
> >> > > > build up the state store as standby tasks and then trigger a
> second
> >> > > > rebalance to actually migrate the active tasks while keeping the
> >> actual
> >> > > > rebalance latency and hence unavailability window to be small (
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=OAmsz8pz4JW%2BayjLqwk04E16G%2FTCF%2BbVk0LNB%2BUJgeY%3D&amp;reserved=0
> >> > > ).
> >> > > > I'd propose we align
> >> > > > KIP-345 along with this idea, and hence do not add the
> >> > expansion_timeout
> >> > > as
> >> > > > part of the protocol layer, but only do that at the application's
> >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> still,
> >> > > > deprecate the "*group.initial.rebalance.delay.ms
> >> > > > <
> >> > > >
> >> > >
> >> >
> >>
> https://nam03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cfa7f68b0cbc94390f04b08d653fa2832%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636788731208857126&amp;sdata=S8Ds6UNM56%2B3rq%2F%2BvYwKdMmzXrH1B5D3ghyjm06hu4g%3D&amp;reserved=0
> >> > > >*"
> >> > > > though as part of this KIP
> >> > > > since we have discussed about its limit and think it is actually
> >> not a
> >> > > very
> >> > > > good design and could be replaced with client-side logic above.
> >> > > >
> >> > > >
> >> > > > 2. I'd like to see your thoughts on the upgrade path for this KIP.
> >> More
> >> > > > specifically, let's say after we have upgraded broker version to
> be
> >> > able
> >> > > to
> >> > > > recognize the new versions of JoinGroup request and the admin
> >> requests,
> >> > > how
> >> > > > should we upgrade the clients and enable static groups? On top of
> my
> >> > head
> >> > > > if we do a rolling bounce in which we set the member.name config
> as
> >> > well
> >> > > > as
> >> > > > optionally increase the session.timeout config when we bounce each
> >> > > > instance, then during this rolling bounces we will have a group
> >> > contained
> >> > > > with both dynamic members and static members. It means that we
> >> should
> >> > > have
> >> > > > the group to allow such scenario (i.e. we cannot reject JoinGroup
> >> > > requests
> >> > > > from dynamic members), and hence the "member.name" -> "member.id"
> >> > > mapping
> >> > > > will only be partial at this scenario. Also could you describe if
> >> the
> >> > > > upgrade to the first version that support this feature would ever
> >> get
> >> > any
> >> > > > benefits, or only the future upgrade path for rolling bounces
> could
> >> get
> >> > > > benefits out of this feature?
> >> > > >
> >> > > > If that's the case and we will do 1) as suggested above, do we
> still
> >> > need
> >> > > > the enableStaticMembership and enableDynamicMembership admin
> >> requests
> >> > any
> >> > > > more? Seems it is not necessary any more as we will only have the
> >> > notion
> >> > > of
> >> > > > "dynamic or static members" that can co-exist in a group while
> >> there no
> >> > > > notion of "dynamic or static groups", and hence these two requests
> >> are
> >> > > not
> >> > > > needed anymore.
> >> > > >
> >> > > >
> >> > > > 3. We need to briefly talk about the implications for ACL as we
> >> > introduce
> >> > > > new admin requests that are related to a specific group.id. For
> >> > example,
> >> > > > we
> >> > > > need to make sure that whoever created the group or joined the
> group
> >> > can
> >> > > > actually send admin requests for the group, otherwise the
> >> application
> >> > > > owners need to bother the Kafka operators on a multi-tenant
> cluster
> >> > every
> >> > > > time they want to send any admin requests for their groups which
> >> would
> >> > be
> >> > > > an operational nightmare.
> >> > > >
> >> > > >
> >> > > > 4. I like Jason's suggestion of adding an optional field for the
> >> list
> >> > of
> >> > > > member names, and I'm wondering if that can be done as part of the
> >> > > > forceStaticRebalance request: i.e. by passing a list of members,
> we
> >> > will
> >> > > > enforce a rebalance immediately since it indicates that some
> static
> >> > > member
> >> > > > will be officially kicked out of the group and some new static
> >> members
> >> > > may
> >> > > > be added. So back to 1.a) above, a static member can only be
> kicked
> >> out
> >> > > of
> >> > > > the group if a) its session (arguably long period of time) has
> timed
> >> > out,
> >> > > > and b) this admin request explicitly state that it is no longer
> >> part of
> >> > > the
> >> > > > group. As for execution I'm fine with keeping it as a future work
> of
> >> > this
> >> > > > KIP if you'd like to make its scope smaller.
> >> > > >
> >> > > > Following are minor comments:
> >> > > >
> >> > > > 5. I'm not sure if we need to include "member.name" as part of
> the
> >> > > > OffsetCommitRequest for fencing purposes, as I think the memberId
> >> plus
> >> > > the
> >> > > > generation number should be sufficient for fencing even with
> static
> >> > > > members.
> >> > > >
> >> > > > 6. As mentioned above, if we agree to do 1) we can get rid of the
> "
> >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> >> > > >
> >> > > >
> >> > > > Guozhang
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > Hey Boyang,
> >> > > > >
> >> > > > > Thanks for the proposal! This is very useful. I have some
> comments
> >> > > below:
> >> > > > >
> >> > > > > 1) The motivation currently explicitly states that the goal is
> to
> >> > > improve
> >> > > > > performance for heavy state application. It seems that the
> >> motivation
> >> > > can
> >> > > > > be stronger with the following use-case. Currently for
> MirrorMaker
> >> > > > cluster
> >> > > > > with e.g. 100 MirrorMaker processes, it will take a long time to
> >> > > rolling
> >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> >> > restart
> >> > > > > will trigger a rebalance which currently pause the consumption
> of
> >> the
> >> > > all
> >> > > > > partitions of the MirrorMaker cluster. With the change stated in
> >> this
> >> > > > > patch, as long as a MirrorMaker can restart within the specified
> >> > > timeout
> >> > > > > (e.g. 2 minutes), then we only need constant number of rebalance
> >> > (e.g.
> >> > > > for
> >> > > > > leader restart) for the entire rolling bounce, which will
> >> > significantly
> >> > > > > improves the availability of the MirrorMaker pipeline. In my
> >> opinion,
> >> > > the
> >> > > > > main benefit of the KIP is to avoid unnecessary rebalance if the
> >> > > consumer
> >> > > > > process can be restarted within soon, which helps performance
> >> even if
> >> > > > > overhead of state shuffling for a given process is small.
> >> > > > >
> >> > > > > 2) In order to simplify the KIP reading, can you follow the
> >> writeup
> >> > > style
> >> > > > > of other KIP (e.g. KIP-98) and list the interface change such as
> >> new
> >> > > > > configs (e.g. registration timeout), new request/response, new
> >> > > > AdminClient
> >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> >> some
> >> > > of
> >> > > > > these are specified in the Proposed Change section which makes
> it
> >> a
> >> > bit
> >> > > > > inconvenient to understand the new interface that will be
> exposed
> >> to
> >> > > > user.
> >> > > > > Explanation of the current two-phase rebalance protocol probably
> >> can
> >> > be
> >> > > > > moved out of public interface section.
> >> > > > >
> >> > > > > 3) There are currently two version of JoinGroupRequest in the
> KIP
> >> and
> >> > > > only
> >> > > > > one of them has field memberId. This seems confusing.
> >> > > > >
> >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> >> rebalance
> >> > > could
> >> > > > > be helpful here, but we will make a call once we finished the
> >> major
> >> > > > > implementation". So this seems to be still an open question in
> the
> >> > > > current
> >> > > > > design. We probably want to agree on this before voting for the
> >> KIP.
> >> > > > >
> >> > > > > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> Can
> >> > you
> >> > > > > specify the name of the config key and the default config value?
> >> > > Possible
> >> > > > > default values include empty string or null (similar to
> >> > transaction.id
> >> > > > in
> >> > > > > producer config).
> >> > > > >
> >> > > > > 6) Regarding the use of the topic "static_member_map" to persist
> >> > member
> >> > > > > name map, currently if consumer coordinator broker goes offline,
> >> > > > rebalance
> >> > > > > is triggered and consumers will try connect to the new
> >> coordinator.
> >> > If
> >> > > > > these consumers can connect to the new coordinator within
> >> > > > > max.poll.interval.ms which by default is 5 minutes, given that
> >> > broker
> >> > > > can
> >> > > > > use a deterministic algorithm to determine the partition ->
> >> > member_name
> >> > > > > mapping, each consumer should get assigned the same set of
> >> partitions
> >> > > > > without requiring state shuffling. So it is not clear whether we
> >> > have a
> >> > > > > strong use-case for this new logic. Can you help clarify what is
> >> the
> >> > > > > benefit of using topic "static_member_map" to persist member
> name
> >> > map?
> >> > > > >
> >> > > > > 7) Regarding the introduction of the expensionTimeoutMs config,
> >> it is
> >> > > > > mentioned that "we are using expansion timeout to replace
> >> rebalance
> >> > > > > timeout, which is configured by max.poll.intervals from client
> >> side,
> >> > > and
> >> > > > > using registration timeout to replace session timeout".
> Currently
> >> the
> >> > > > > default max.poll.interval.ms is configured to be 5 minutes and
> >> there
> >> > > > will
> >> > > > > be only one rebalance if all new consumers can join within 5
> >> minutes.
> >> > > So
> >> > > > it
> >> > > > > is not clear whether we have a strong use-case for this new
> >> config.
> >> > Can
> >> > > > you
> >> > > > > explain what is the benefit of introducing this new config?
> >> > > > >
> >> > > > > 8) It is mentioned that "To distinguish between previous version
> >> of
> >> > > > > protocol, we will also increase the join group request version
> to
> >> v4
> >> > > when
> >> > > > > MEMBER_NAME is set" and "If the broker version is not the latest
> >> (<
> >> > > v4),
> >> > > > > the join group request shall be downgraded to v3 without setting
> >> the
> >> > > > member
> >> > > > > Id". It is probably simpler to just say that this feature is
> >> enabled
> >> > if
> >> > > > > JoinGroupRequest V4 is supported on both client and broker and
> >> > > > MEMBER_NAME
> >> > > > > is configured with non-empty string.
> >> > > > >
> >> > > > > 9) It is mentioned that broker may return
> >> NO_STATIC_MEMBER_INFO_SET
> >> > > error
> >> > > > > in OffsetCommitResponse for "commit requests under static
> >> > membership".
> >> > > > Can
> >> > > > > you clarify how broker determines whether the commit request is
> >> under
> >> > > > > static membership?
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Dong
> >> > > > >
> >> > > >
> >> > > >
> >> > > > --
> >> > > > -- Guozhang
> >> > > >
> >> > >
> >> >
> >> >
> >> > --
> >> > -Regards,
> >> > Mayuresh R. Gharat
> >> > (862) 250-7125
> >> >
> >>
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Jason Gustafson <ja...@confluent.io>.
Hi Boyang,

Thanks for the updates. Looks like we're headed in the right direction and
clearly the interest that this KIP is receiving shows how strong the
motivation is!

I have a few questions:

1. This may be the same thing that Mayuresh is asking about. I think the
suggestion in the KIP is that if a consumer sends JoinGroup with a member
name, but no member id, then we will return the current member id
associated with that name. It seems in this case that we wouldn't be able
to protect from having two consumers active with the same configured
member.name? For example, imagine that we had a consumer with member.name=A
which is assigned member.id=1. Suppose it becomes a zombie and a new
instance starts up with member.name=A. If it is also assigned member.id=1,
then how can we detect the zombie if it comes back to life? Both instances
will have the same member.id.

The goal is to avoid a rebalance on a rolling restart, but we still need to
fence previous members. I am wondering if we can generate a new member.id
every time we receive a request from a static member with an unknown member
id. If the old instance with the same member.name attempts any operation,
then it will be fenced with an UNKNOWN_MEMBER_ID error. As long as the
subscription of the new instance hasn't changed, then we can skip the
rebalance and return the current assignment without forcing a rebalance.

The trick to making this work is in the error handling of the zombie
consumer. If the zombie simply resets its member.id and rejoins to get a
new one upon receiving the UNKNOWN_MEMBER_ID error, then it would end up
fencing the new member. We want to avoid this. There needs to be an
expectation for static members that the member.id of a static member will
not be changed except when a new member with the same member.name joins the
group. Then we can treat UNKNOWN_MEMBER_ID as a fatal error for consumers
with static member names.

2. The mechanics of the ConsumerRebalance API seem unclear to me. As far as
I understand it, it is used for scaling down a consumer group and somehow
bypasses normal session timeout expiration. I am wondering how critical
this piece is and whether we can leave it for future work. If not, then it
would be helpful to elaborate on its implementation. How would the
coordinator know which members to kick out of the group?

3. I've been holding back on mentioning this, but I think we should
reconsider the name `member.name`. I think we want something that suggests
its expectation of uniqueness in the group. How about `group.instance.id`
to go along with `group.id`?

Thanks,
Jason



On Mon, Nov 26, 2018 at 10:18 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> Hi Boyang,
>
> Thanks a lot for replying to all the queries and discussions here, so
> patiently.
> Really appreciate it.
>
> Had a few questions and suggestions after rereading the current version of
> the KIP :
>
>
>    1. Do you intend to have member.id is a static config like member.name
>    after KIP-345 and KIP-394?
>    2. Regarding "On client side, we add a new config called MEMBER_NAME in
>    ConsumerConfig. On consumer service init, if the MEMBER_NAME config is
> set,
>    we will put it in the initial join group request to identify itself as a
>    static member (static membership); otherwise, we will still send
>    UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID (dynamic
>    membership)."
>       - What is the value of member_id sent in the first JoinGroupRequest
>       when member_name is set (using static rebalance)? Is it
> UNKNOW_MEMBER_ID?
>    3. Regarding "we are requiring member.id (if not unknown) to match the
>    value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge case
>    that if we could have members with the same `member.name` (for example
>    mis-configured instances with a valid member.id but added a used member
>    name on runtime). When member name has duplicates, we could refuse join
>    request from members with an outdated `member.id` (since we update the
>    mapping upon each join group request). In an edge case where the client
>    hits this exception in the response, it is suggesting that some other
>    consumer takes its spot."
>       - The part of "some other consumer takes the spot" would be
>       intentional, right? Also when you say " The edge case that if we
>       could have members with the same `member.name` (for example
>       mis-configured instances *with a valid member.id <http://member.id>
> *but
>       added a used member name on runtime).", what do you mean by *valid
>       member id* here? Does it mean that there exist a mapping of
>       member.name to member.id like *MemberA -> id1* on the
>       GroupCoordinator and this consumer is trying to join with *
> member.name
>       <http://member.name> = MemberB and member.id <http://member.id> =
> id1 *
>       ?
>    4. Depending on your explanation for point 2 and the point 3 above
>    regarding returning back MEMBER_ID_MISMATCH on having a matching
>    member_name but unknown member_id, if the consumer sends
> "UNKNOW_MEMBER_ID"
>    on the first JoinGroupRequest and relies on the GroupCoordinator to
> give it
>    a member_id, is the consumer suppose to remember member_id for
>    joinGroupRequests? If yes, how are restarts handled?
>    5. Regarding "So in summary, *the member will only be removed due to
>    session timeout*. We shall remove it from both in-memory static member
>    name mapping and member list."
>       - If the rebalance is invoked manually using the the admin apis, how
>       long should the group coordinator wait for the members of the
> group to send
>       a JoinGroupRequest for participating in the rebalance? How is a
> lagging
>       consumer handled?
>    6. Another detail to take care is that we need to automatically take the
>    hash of group id so that we know which broker to send this request to.
>       - I assume this should be same as the way we find the coordinator,
>       today right? If yes, should we specify it in the KIP ?
>    7. Are there any specific failure scenarios when you say "other
>    potential failure cases."? It would be good to mention them explicitly,
> if
>    you think there are any.
>    8. It would be good to have a rollback plan as you have for roll forward
>    in the KIP.
>
> Thanks,
>
> Mayuresh
>
> On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <
> gharatmayuresh15@gmail.com>
> wrote:
>
> > Hi Boyang,
> >
> > Do you have a discuss thread for KIP-394 that you mentioned here ?
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> >> Hey Dong, thanks for the follow-up here!
> >>
> >>
> >> 1) It is not very clear to the user what is the difference between
> >> member.name and client.id as both seems to be used to identify the
> >> consumer. I am wondering if it would be more intuitive to name it
> >> group.member.name (preferred choice since it matches the current
> group.id
> >> config name) or rebalance.member.name to explicitly show that the id is
> >> solely used for rebalance.
> >> Great question. I feel `member.name` is enough to explain itself, it
> >> seems not very
> >> helpful to make the config name longer. Comparing `name` with `id` gives
> >> user the
> >> impression that they have the control over it with customized rule than
> >> library decided.
> >>
> >> 2) In the interface change section it is said that
> >> GroupMaxSessionTimeoutMs
> >> will be changed to 30 minutes. It seems to suggest that we will change
> the
> >> default value of this config. It does not seem necessary to increase the
> >> time of consumer failure detection when user doesn't use static
> >> membership.
> >> Also, say static membership is enabled, then this default config change
> >> will cause a partition to be unavailable for consumption for 30 minutes
> if
> >> there is hard consumer failure, which seems to be worse experience than
> >> having unnecessary rebalance (when this timeout is small), particularly
> >> for
> >> new users of Kafka. Could you explain more why we should make this
> change?
> >> We are not changing the default session timeout value. We are just
> >> changing the
> >> cap we are enforcing on the session timeout max value. So this change is
> >> not affecting
> >> what kind of membership end user is using, and loosing the cap is giving
> >> end user
> >> more flexibility on trade-off between liveness and stability.
> >>
> >> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> >> into one error? It seems that these two errors are currently handled by
> >> the
> >> consumer in the same way. And we don't also don't expect
> >> MEMBER_ID_MISMATCH
> >> to happen. Thus it is not clear what is the benefit of having two
> errors.
> >> I agree that we should remove DUPLICATE_STATIC_MEMBER error because with
> >> the KIP-394<
> >>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member+id+for+initial+join+group+request
> >> >
> >> we will automatically fence all join requests with UNKNOWN_MEMBER_ID.
> >>
> >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> contains
> >> member name which is already in the consumer group, however the member
> id
> >> was missing". After a consumer is restarted, it will send a
> >> JoinGroupRequest with an existing memberName (as the coordinator has not
> >> expired this member from the memory) and memberId
> >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> >> across consumer restart in the consumer side). Does it mean that
> >> JoinGroupRequest from a newly restarted consumer will always be rejected
> >> until the sessionTimeoutMs has passed?
> >> Same answer as question 3). This part of the logic shall be removed from
> >> the proposal.
> >>
> >> 5) It seems that we always add two methods to the interface
> >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> the
> >> other without option. Could this be specified in the interface change
> >> section?
> >> Sounds good! Added both methods.
> >>
> >> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> >> rebalance? If so, we probably want to specify the command line tool
> >> interface similar to
> >>
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
> >> .
> >> Added the script.
> >>
> >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> >> "invokeConsumerRebalance"? It is not very clear what is the extra
> meaning
> >> of world "force" as compared to "trigger" or "invoke". And it seems
> >> simpler
> >> to allows this API to trigger rebalance regardless of whether consumer
> is
> >> configured with memberName.
> >> Sounds good. Right now I feel for both static and dynamic membership it
> is
> >> more manageable to introduce the consumer rebalance method through admin
> >> client API.
> >>
> >> 8) It is not very clear how the newly added AdminClient API trigger
> >> rebalance. For example, does it send request? Can this be explained in
> the
> >> KIP?
> >>
> >> Sure, I will add more details to the API.
> >>
> >>
> >> Thanks again for the helpful suggestions!
> >>
> >>
> >> Best,
> >> Boyang
> >>
> >> ________________________________
> >> From: Dong Lin <li...@gmail.com>
> >> Sent: Saturday, November 24, 2018 2:54 PM
> >> To: dev
> >> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> specifying member id
> >>
> >> Hey Boyang,
> >>
> >> Thanks for the update! Here are some followup comments:
> >>
> >> 1) It is not very clear to the user what is the difference between
> >> member.name and client.id as both seems to be used to identify the
> >> consumer. I am wondering if it would be more intuitive to name it
> >> group.member.name (preferred choice since it matches the current
> group.id
> >> config name) or rebalance.member.name to explicitly show that the id is
> >> solely used for rebalance.
> >>
> >> 2) In the interface change section it is said that
> >> GroupMaxSessionTimeoutMs
> >> will be changed to 30 minutes. It seems to suggest that we will change
> the
> >> default value of this config. It does not seem necessary to increase the
> >> time of consumer failure detection when user doesn't use static
> >> membership.
> >> Also, say static membership is enabled, then this default config change
> >> will cause a partition to be unavailable for consumption for 30 minutes
> if
> >> there is hard consumer failure, which seems to be worse experience than
> >> having unnecessary rebalance (when this timeout is small), particularly
> >> for
> >> new users of Kafka. Could you explain more why we should make this
> change?
> >>
> >> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> >> into one error? It seems that these two errors are currently handled by
> >> the
> >> consumer in the same way. And we don't also don't expect
> >> MEMBER_ID_MISMATCH
> >> to happen. Thus it is not clear what is the benefit of having two
> errors.
> >>
> >> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group
> contains
> >> member name which is already in the consumer group, however the member
> id
> >> was missing". After a consumer is restarted, it will send a
> >> JoinGroupRequest with an existing memberName (as the coordinator has not
> >> expired this member from the memory) and memberId
> >> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> >> across consumer restart in the consumer side). Does it mean that
> >> JoinGroupRequest from a newly restarted consumer will always be rejected
> >> until the sessionTimeoutMs has passed?
> >>
> >> 5) It seems that we always add two methods to the interface
> >> org.apache.kafka.clients.admin.AdminClient.java, one with options and
> the
> >> other without option. Could this be specified in the interface change
> >> section?
> >>
> >> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> >> rebalance? If so, we probably want to specify the command line tool
> >> interface similar to
> >>
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
> >> .
> >>
> >> 7) Would it be simpler to replace name "forceStaticRebalance" with
> >> "invokeConsumerRebalance"? It is not very clear what is the extra
> meaning
> >> of world "force" as compared to "trigger" or "invoke". And it seems
> >> simpler
> >> to allows this API to trigger rebalance regardless of whether consumer
> is
> >> configured with memberName.
> >>
> >> 8) It is not very clear how the newly added AdminClient API trigger
> >> rebalance. For example, does it send request? Can this be explained in
> the
> >> KIP?
> >>
> >> Thanks,
> >> Dong
> >>
> >>
> >> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com>
> wrote:
> >>
> >> > Hey Mayuresh,
> >> >
> >> >
> >> > thanks for your feedbacks! I will try do another checklist here.
> >> >
> >> >
> >> > > By this you mean, even if the application has not called
> >> > > KafkaConsumer.poll() within session timeout, it will not be sending
> >> the
> >> > > LeaveGroup request, right?
> >> >
> >> > Yep it's true, we will prevent client from sending leave group request
> >> > when they are set with `member.name`.
> >> >
> >> >
> >> > > When is the member.name removed from this map?
> >> > Good question, we will only kick off member due to session timeout
> >> within
> >> > static membership. Let me update the KIP to clearly assert that.
> >> >
> >> > > How is this case (missing member id) handled on the client side?
> What
> >> is
> >> > the application that
> >> > > is using the KafkaConsumer suppose to do in this scenario?
> >> > I have extended the two exceptions within join group response V4.
> >> > Basically I define both corresponding actions to be immediate failing
> >> > client application, because so far it is unknown what kind of client
> >> issue
> >> > could trigger them. After the first version, we will keep enhance the
> >> error
> >> > handling logic!
> >> >
> >> > > This would mean that it might take more time to detect unowned topic
> >> > > partitions and may cause delay for applications that perform data
> >> > mirroring
> >> > > tasks. I discussed this with our sre and we have a suggestion to
> make
> >> > here
> >> > > as listed below separately.
> >> > The goal of extending session timeout cap is for users with good
> client
> >> > side monitoring tools that could auto-heal the dead consumers very
> >> fast. So
> >> > it is optional (and personal) to extend session timeout to a
> reasonable
> >> > number with different client scenarios.
> >> >
> >> > > you meant remove unjoined members of the group, right ?
> >> > Yep, there is a typo. Thanks for catching this!
> >> >
> >> > > What do you mean by " Internally we would optimize this logic by
> >> having
> >> > > rebalance timeout only in charge of stopping prepare rebalance
> stage,
> >> > > without removing non-responsive members immediately." There would
> not
> >> be
> >> > a
> >> > > full rebalance if the lagging consumer sent a JoinGroup request
> later,
> >> > > right ? If yes, can you highlight this in the KIP ?
> >> > No, there won't be. We want to limit the rebalance timeout
> functionality
> >> > to only use as a timer to
> >> > end prepare rebalance stage. This way, late joining static members
> will
> >> > not trigger further rebalance
> >> > as long as they are within session timeout. I added your highlight to
> >> the
> >> > KIP!
> >> >
> >> > > The KIP talks about scale up scenario but its not quite clear how we
> >> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> >> > status
> >> > > "learner" ?. Can you shed more light on how this is handled in the
> >> KIP,
> >> > if
> >> > > its handled?
> >> > Updated the KIP: we shall not cover scale up case in 345, because we
> >> > believe client side could
> >> > better handle this logic.
> >> >
> >> > > I think Jason had brought this up earlier about having a way to say
> >> how
> >> > > many members/consumer hosts are you choosing to be in the consumer
> >> group.
> >> > > If we can do this, then in case of mirroring applications we can do
> >> this
> >> > :
> >> > > Lets say we have a mirroring application that consumes from Kafka
> >> cluster
> >> > > A and produces to Kafka cluster B.
> >> > > Depending on the data and the Kafka cluster configuration, Kafka
> >> service
> >> > > providers can set a mirroring group saying that it will take, for
> >> example
> >> > > 300 consumer hosts/members to achieve the desired throughput and
> >> latency
> >> > > for mirroring and can have additional 10 consumer hosts as spare in
> >> the
> >> > > same group.
> >> > > So when the first 300 members/consumers to join the group will start
> >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> >> > > The remaining 10 consumer members can sit idle.
> >> > > The moment one of the consumer (for example: consumer number 54)
> from
> >> the
> >> > > first 300 members go out of the group (crossed session timeout), it
> >> (the
> >> > > groupCoordinator) can just assign the topicPartitions from the
> >> consumer
> >> > > member 54 to one of the spare hosts.
> >> > > Once the consumer member 54 comes back up, it can start as being a
> >> part
> >> > of
> >> > > the spare pool.
> >> > > This enables us to have lower session timeouts and low latency
> >> mirroring,
> >> > > in cases where the service providers are OK with having spare hosts.
> >> > > This would mean that we would tolerate n consumer members leaving
> and
> >> > > rejoining the group and still provide low latency as long as n <=
> >> number
> >> > of
> >> > > spare consumers.
> >> > > If there are no spare host available, we can get back to the idea as
> >> > > described in the KIP.
> >> > Great idea! In fact on top of static membership we could later
> introduce
> >> > APIs to set hard-coded
> >> > client ids to the group and replace the dead host, or as you proposed
> to
> >> > define spare host as
> >> > what I understood as hot backup. I will put both Jason and your
> >> > suggestions into a separate section
> >> > called "Future works". Note that this spare host idea may be also
> >> solvable
> >> > through rebalance protocol
> >> > IMO.
> >> >
> >> > Thank you again for the great feedback!
> >> >
> >> > Boyang
> >> > ________________________________
> >> > From: Boyang Chen <bc...@outlook.com>
> >> > Sent: Thursday, November 22, 2018 3:39 PM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> > specifying member id
> >> >
> >> > Hey Dong, sorry for missing your message. I couldn't find your email
> on
> >> my
> >> > thread, so I will just do a checklist here!
> >> >
> >> >
> >> > 1) The motivation currently explicitly states that the goal is to
> >> improve
> >> >
> >> > performance for heavy state application. It seems that the motivation
> >> can
> >> >
> >> > be stronger with the following use-case. Currently for MirrorMaker
> >> cluster
> >> >
> >> > with e.g. 100 MirrorMaker processes, it will take a long time to
> rolling
> >> >
> >> > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> restart
> >> >
> >> > will trigger a rebalance which currently pause the consumption of the
> >> all
> >> >
> >> > partitions of the MirrorMaker cluster. With the change stated in this
> >> >
> >> > patch, as long as a MirrorMaker can restart within the specified
> timeout
> >> >
> >> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> >> for
> >> >
> >> > leader restart) for the entire rolling bounce, which will
> significantly
> >> >
> >> > improves the availability of the MirrorMaker pipeline. In my opinion,
> >> the
> >> >
> >> > main benefit of the KIP is to avoid unnecessary rebalance if the
> >> consumer
> >> >
> >> > process can be restarted within soon, which helps performance even if
> >> >
> >> > overhead of state shuffling for a given process is small.
> >> >
> >> > I just rephrased this part and added it to the KIP. Thanks for making
> >> the
> >> > motivation more solid!
> >> >
> >> > 2) In order to simplify the KIP reading, can you follow the writeup
> >> style
> >> > of other KIP (e.g. KIP-98) and list the interface change such as new
> >> > configs (e.g. registration timeout), new request/response, new
> >> AdminClient
> >> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> of
> >> > these are specified in the Proposed Change section which makes it a
> bit
> >> > inconvenient to understand the new interface that will be exposed to
> >> user.
> >> > Explanation of the current two-phase rebalance protocol probably can
> be
> >> > moved out of public interface section.
> >> > This is a great suggestion! I just consolidated all the public API
> >> > changes, and the whole KIP
> >> > looks much more organized!
> >> >
> >> > 3) There are currently two version of JoinGroupRequest in the KIP and
> >> only
> >> > one of them has field memberId. This seems confusing.
> >> > Yep, I already found this issue and fixed it.
> >> >
> >> > 4) It is mentioned in the KIP that "An admin API to force rebalance
> >> could
> >> > be helpful here, but we will make a call once we finished the major
> >> > implementation". So this seems to be still an open question in the
> >> current
> >> > design. We probably want to agree on this before voting for the KIP.
> >> > We have finalized the idea that this API is needed.
> >> >
> >> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> >> > specify the name of the config key and the default config value?
> >> Possible
> >> > default values include empty string or null (similar to
> transaction.id<
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=kkCtxHpsAZpbJZpUc52dtv6ac8UJOx6CQlts3CPjDh8%3D&amp;reserved=0
> >> >
> >> > in
> >> > producer config).
> >> > I have defined the `member.name` in "New configuration" section.
> >> >
> >> > 6) Regarding the use of the topic "static_member_map" to persist
> member
> >> > name map, currently if consumer coordinator broker goes offline,
> >> rebalance
> >> > is triggered and consumers will try connect to the new coordinator. If
> >> > these consumers can connect to the new coordinator within
> >> > max.poll.interval.ms<
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
> >> >
> >> > which by default is 5 minutes, given that broker can
> >> > use a deterministic algorithm to determine the partition ->
> member_name
> >> > mapping, each consumer should get assigned the same set of partitions
> >> > without requiring state shuffling. So it is not clear whether we have
> a
> >> > strong use-case for this new logic. Can you help clarify what is the
> >> > benefit of using topic "static_member_map" to persist member name map?
> >> > I have discussed with Guozhang offline, and I believe reusing the
> >> current
> >> > `_consumer_offsets`
> >> > topic is a better and unified solution.
> >> >
> >> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> >> > mentioned that "we are using expansion timeout to replace rebalance
> >> > timeout, which is configured by max.poll.intervals from client side,
> and
> >> > using registration timeout to replace session timeout". Currently the
> >> > default max.poll.interval.ms<
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
> >> >
> >> > is configured to be 5 minutes and there will
> >> > be only one rebalance if all new consumers can join within 5 minutes.
> >> So it
> >> > is not clear whether we have a strong use-case for this new config.
> Can
> >> you
> >> > explain what is the benefit of introducing this new config?
> >> > Previously our goal is to use expansion timeout as a workaround for
> >> > triggering multiple
> >> > rebalances when scaling up members are not joining at the same time.
> It
> >> is
> >> > decided to
> >> > be addressed by client side protocol change, so we will not introduce
> >> > expansion timeout.
> >> >
> >> > 8) It is mentioned that "To distinguish between previous version of
> >> > protocol, we will also increase the join group request version to v4
> >> when
> >> > MEMBER_NAME is set" and "If the broker version is not the latest (<
> v4),
> >> > the join group request shall be downgraded to v3 without setting the
> >> member
> >> > Id". It is probably simpler to just say that this feature is enabled
> if
> >> > JoinGroupRequest V4 is supported on both client and broker and
> >> MEMBER_NAME
> >> > is configured with non-empty string.
> >> > Yep, addressed this!
> >> >
> >> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> >> error
> >> > in OffsetCommitResponse for "commit requests under static membership".
> >> Can
> >> > you clarify how broker determines whether the commit request is under
> >> > static membership?
> >> >
> >> > We have agreed that commit request shouldn't be affected by the new
> >> > membership, thus
> >> > removing it here. Thanks for catching this!
> >> >
> >> > Let me know if you have further suggestions or concerns. Thank you for
> >> > your valuable feedback
> >> > to help me design the KIP better! (And I will try to address your
> >> > feedbacks in next round Mayuresh ??)
> >> >
> >> > Best,
> >> > Boyang
> >> > ________________________________
> >> > From: Mayuresh Gharat <gh...@gmail.com>
> >> > Sent: Wednesday, November 21, 2018 7:50 AM
> >> > To: dev@kafka.apache.org
> >> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> >> > specifying member id
> >> >
> >> > Hi Boyang,
> >> >
> >> > Thanks for updating the KIP. This is a step good direction for
> stateful
> >> > applications and also mirroring applications whose latency is affected
> >> due
> >> > to the rebalance issues that we have today.
> >> >
> >> > I had a few questions on the current version of the KIP :
> >> > For the effectiveness of the KIP, consumer with member.name set will
> >> *not
> >> > send leave group request* when they go offline
> >> >
> >> > > By this you mean, even if the application has not called
> >> > > KafkaConsumer.poll() within session timeout, it will not be sending
> >> the
> >> > > LeaveGroup request, right?
> >> > >
> >> >
> >> > Broker will maintain an in-memory mapping of {member.name ? member.id
> }
> >> to
> >> > track member uniqueness.
> >> >
> >> > > When is the member.name removed from this map?
> >> > >
> >> >
> >> > Member.id must be set if the *member.name <
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=aSTzJlF4GPFGKhi5K7MPZozMn67718GWpqEYxFay%2BZs%3D&amp;reserved=0
> >> >
> >> > *is already
> >> > within the map. Otherwise reply MISSING_MEMBER_ID
> >> >
> >> > > How is this case handled on the client side? What is the application
> >> that
> >> > > is using the KafkaConsumer suppose to do in this scenario?
> >> > >
> >> >
> >> > Session timeout is the timeout we will trigger rebalance when a member
> >> goes
> >> > offline for too long (not sending heartbeat request). To make static
> >> > membership effective, we should increase the default max session
> >> timeout to
> >> > 30 min so that end user could config it freely.
> >> >
> >> > > This would mean that it might take more time to detect unowned topic
> >> > > partitions and may cause delay for applications that perform data
> >> > mirroring
> >> > > tasks. I discussed this with our sre and we have a suggestion to
> make
> >> > here
> >> > > as listed below separately.
> >> > >
> >> >
> >> > Currently there is a config called *rebalance timeout* which is
> >> configured
> >> > by consumer *max.poll.intervals*. The reason we set it to poll
> interval
> >> is
> >> > because consumer could only send request within the call of poll() and
> >> we
> >> > want to wait sufficient time for the join group request. When reaching
> >> > rebalance timeout, the group will move towards completingRebalance
> stage
> >> > and remove unjoined groups
> >> >
> >> > > you meant remove unjoined members of the group, right ?
> >> > >
> >> >
> >> > Currently there is a config called *rebalance timeout* which is
> >> configured
> >> > by consumer *max.poll.intervals*. The reason we set it to poll
> interval
> >> is
> >> > because consumer could only send request within the call of poll() and
> >> we
> >> > want to wait sufficient time for the join group request. When reaching
> >> > rebalance timeout, the group will move towards completingRebalance
> stage
> >> > and remove unjoined groups. This is actually conflicting with the
> >> design of
> >> > static membership, because those temporarily unavailable members will
> >> > potentially reattempt the join group and trigger extra rebalances.
> >> > Internally we would optimize this logic by having rebalance timeout
> >> only in
> >> > charge of stopping prepare rebalance stage, without removing
> >> non-responsive
> >> > members immediately.
> >> >
> >> > > What do you mean by " Internally we would optimize this logic by
> >> having
> >> > > rebalance timeout only in charge of stopping prepare rebalance
> stage,
> >> > > without removing non-responsive members immediately." There would
> not
> >> be
> >> > a
> >> > > full rebalance if the lagging consumer sent a JoinGroup request
> later,
> >> > > right ? If yes, can you highlight this in the KIP ?
> >> > >
> >> >
> >> > Scale Up
> >> >
> >> > > The KIP talks about scale up scenario but its not quite clear how we
> >> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> >> > status
> >> > > "learner" ?. Can you shed more light on how this is handled in the
> >> KIP,
> >> > if
> >> > > its handled?
> >> > >
> >> >
> >> >
> >> > *Discussion*
> >> > Larger session timeouts causing latency rise for getting data for
> >> un-owned
> >> > topic partitions :
> >> >
> >> > > I think Jason had brought this up earlier about having a way to say
> >> how
> >> > > many members/consumer hosts are you choosing to be in the consumer
> >> group.
> >> > > If we can do this, then in case of mirroring applications we can do
> >> this
> >> > :
> >> > > Lets say we have a mirroring application that consumes from Kafka
> >> cluster
> >> > > A and produces to Kafka cluster B.
> >> > > Depending on the data and the Kafka cluster configuration, Kafka
> >> service
> >> > > providers can set a mirroring group saying that it will take, for
> >> example
> >> > > 300 consumer hosts/members to achieve the desired throughput and
> >> latency
> >> > > for mirroring and can have additional 10 consumer hosts as spare in
> >> the
> >> > > same group.
> >> > > So when the first 300 members/consumers to join the group will start
> >> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> >> > > The remaining 10 consumer members can sit idle.
> >> > > The moment one of the consumer (for example: consumer number 54)
> from
> >> the
> >> > > first 300 members go out of the group (crossed session timeout), it
> >> (the
> >> > > groupCoordinator) can just assign the topicPartitions from the
> >> consumer
> >> > > member 54 to one of the spare hosts.
> >> > > Once the consumer member 54 comes back up, it can start as being a
> >> part
> >> > of
> >> > > the spare pool.
> >> > > This enables us to have lower session timeouts and low latency
> >> mirroring,
> >> > > in cases where the service providers are OK with having spare hosts.
> >> > > This would mean that we would tolerate n consumer members leaving
> and
> >> > > rejoining the group and still provide low latency as long as n <=
> >> number
> >> > of
> >> > > spare consumers.
> >> > > If there are no spare host available, we can get back to the idea as
> >> > > described in the KIP.
> >> > >
> >> >
> >> > Thanks,
> >> >
> >> > Mayuresh
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> >> > konstantine@confluent.io> wrote:
> >> >
> >> > > Hi Boyang.
> >> > >
> >> > > Thanks for preparing this KIP! It is making good progress and will
> be
> >> a
> >> > > great improvement for stateful Kafka applications.
> >> > >
> >> > > Apologies for my late reply, I was away for a while. Lots of great
> >> > comments
> >> > > so far, so I'll probably second most of them in what I suggest below
> >> at
> >> > > this point.
> >> > >
> >> > > When I first read the KIP, I wanted to start at the end with
> something
> >> > that
> >> > > wasn't highlighted a lot. That was the topic related to handling
> >> > duplicate
> >> > > members. I see now that the initial suggestion of handling this
> >> situation
> >> > > during offset commit has been removed, and I agree with that. Issues
> >> > > related to membership seem to be handled better when the member
> joins
> >> the
> >> > > group rather than when it tries to commit offsets. This also
> >> simplifies
> >> > how
> >> > > many request types need to change in order to incorporate the new
> >> member
> >> > > name field.
> >> > >
> >> > > I also agree with what Jason and Guozhang have said regarding
> >> timeouts.
> >> > > Although semantically, it's easier to think of every operation
> having
> >> its
> >> > > own timeout, operationally this can become a burden. Thus,
> >> consolidation
> >> > > seems preferable here. The definition of embedded protocols on top
> of
> >> the
> >> > > base group membership protocol for rebalancing gives enough
> >> flexibility
> >> > to
> >> > > address such needs in each client component separately.
> >> > >
> >> > > Finally, some minor comments:
> >> > > In a few places the new/proposed changes are referred to as
> "current".
> >> > > Which is a bit confusing considering that there is a protocol in
> place
> >> > > already, and by "current" someone might understand the existing one.
> >> I'd
> >> > > recommend using new/proposed or equivalent when referring to changes
> >> > > introduced with KIP-345 and current/existing or equivalent when
> >> referring
> >> > > to existing behavior.
> >> > >
> >> > > There's the following sentence in the "Public Interfaces" section:
> >> > > "Since for many stateful consumer/stream applications, the state
> >> > shuffling
> >> > > is more painful than short time partial unavailability."
> >> > > However, my understanding is that the changes proposed with KIP-345
> >> will
> >> > > not exploit any partial availability. A suggestion for dealing with
> >> > > temporary imbalances has been made in "Incremental Cooperative
> >> > Rebalancing"
> >> > > which can work well with KIP-345, but here I don't see proposed
> >> changes
> >> > > that suggest that some resources (e.g. partitions) will keep being
> >> used
> >> > > while others will not be utilized. Thus, you might want to adjust
> this
> >> > > sentence. Correct me if I'm missing something related to that.
> >> > >
> >> > > In the rejected alternatives, under point 2) I read "we can copy the
> >> > member
> >> > > id to the config files". I believe it means to say "member name"
> >> unless
> >> > I'm
> >> > > missing something about reusing member ids. Also below I read: "By
> >> > allowing
> >> > > consumers to optionally specifying a member id" which probably
> implies
> >> > > "member name" again. In a sense this section highlights a potential
> >> > > confusion between member name and member id. I wonder if we could
> >> come up
> >> > > with a better term for the new field. StaticTag, StaticLabel, or
> even
> >> > > StaticName are some suggestions that could potentially help with
> >> > confusion
> >> > > between MemberId and MemberName and what corresponds to what. But I
> >> > > wouldn't like to disrupt the discussion with naming conventions too
> >> much
> >> > at
> >> > > this point. I just mention it here as a thought.
> >> > >
> >> > > Looking forward to see the final details of this KIP. Great work so
> >> far!
> >> > >
> >> > > Konstantine
> >> > >
> >> > >
> >> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com>
> >> wrote:
> >> > >
> >> > > > Thanks Guozhang for the great summary here, and I have been
> >> following
> >> > up
> >> > > > the action items here.
> >> > > >
> >> > > >
> >> > > >   1.  I already updated the KIP to remove the expansion timeout
> and
> >> > > > registration timeout. Great to see them being addressed in client
> >> side!
> >> > > >   2.  I double checked the design and I believe that it is ok to
> >> have
> >> > > both
> >> > > > static member and dynamic member co-exist in the same group. So
> the
> >> > > upgrade
> >> > > > shouldn't be destructive and we are removing the two membership
> >> > protocol
> >> > > > switching APIs.
> >> > > >   3.  I only have question about this one. I'm still reading the
> >> > > KafkaApis
> >> > > > code here. Should I just use the same authorization logic for
> >> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> >> > > >   4.  I'm very excited to see this work with K8! Like you
> suggested,
> >> > this
> >> > > > feature could be better addressed in a separate KIP because it is
> >> > pretty
> >> > > > independent. I could start drafting the KIP once the current
> >> proposal
> >> > is
> >> > > > approved.
> >> > > >   5.  I believe that we don't need fencing in offset commit
> request,
> >> > > since
> >> > > > duplicate member.name issue could be handled by join group
> >> request. We
> >> > > > shall reject join group with known member name but no member id
> >> (which
> >> > > > means we already have an active member using this identity).
> >> > > >   6.  I agree to remove that internal config once we move forward
> >> with
> >> > > > static membership. And I already removed the entire section from
> the
> >> > KIP.
> >> > > >
> >> > > > Let me know if you have other concerns.
> >> > > >
> >> > > > Best,
> >> > > > Boyang
> >> > > > ________________________________
> >> > > > From: Guozhang Wang <wa...@gmail.com>
> >> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> >> > > > To: dev
> >> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> >> by
> >> > > > specifying member id
> >> > > >
> >> > > > Hello Boyang,
> >> > > >
> >> > > > Thanks a lot for the KIP! It is a great write-up and I appreciate
> >> your
> >> > > > patience answering to the feedbacks from the community. I'd like
> to
> >> add
> >> > > my
> >> > > > 2cents here:
> >> > > >
> >> > > > 1. By introducing another two timeout configs,
> registration_timeout
> >> and
> >> > > > expansion_timeout, we are effectively having four timeout configs:
> >> > > session
> >> > > > timeout, rebalance timeout (configured as "max.poll.interval.ms"
> on
> >> > > client
> >> > > > side), and these two. Interplaying these timeout configs can be
> >> quite
> >> > > hard
> >> > > > for users with such complexity, and hence I'm wondering if we can
> >> > > simplify
> >> > > > the situation with as less possible timeout configs as possible.
> >> Here
> >> > is
> >> > > a
> >> > > > concrete suggestion I'd like propose:
> >> > > >
> >> > > > 1.a) Instead of introducing a registration_timeout in addition to
> >> the
> >> > > > session_timeout for static members, we can just reuse the
> >> > session_timeout
> >> > > > and ask users to set it to a larger value when they are upgrading
> a
> >> > > dynamic
> >> > > > client to a static client by setting the "member.name" at the
> same
> >> > time.
> >> > > > By
> >> > > > default, the broker-side min.session.timeout is 6 seconds and
> >> > > > max.session.timeout is 5 minutes, which seems reasonable to me (we
> >> can
> >> > of
> >> > > > course modify this broker config to enlarge the valid interval if
> we
> >> > want
> >> > > > in practice). And then we should also consider removing the
> >> condition
> >> > for
> >> > > > marking a client as failed if the rebalance timeout has reached
> >> while
> >> > the
> >> > > > JoinGroup was not received, so that the semantics of
> session_timeout
> >> > and
> >> > > > rebalance_timeout are totally separated: the former is only used
> to
> >> > > > determine if a consumer member of the group should be marked as
> >> failed
> >> > > and
> >> > > > kicked out of the group, and the latter is only used to determine
> >> the
> >> > > > longest time coordinator should wait for PREPARE_REBALANCE phase.
> In
> >> > > other
> >> > > > words if a member did not send the JoinGroup in time of the
> >> > > > rebalance_timeout, we still include it in the new generation of
> the
> >> > group
> >> > > > and use its old subscription info to send to leader for
> assignment.
> >> > Later
> >> > > > if the member came back with HeartBeat request, we can still
> follow
> >> the
> >> > > > normal path to bring it to the latest generation while checking
> that
> >> > its
> >> > > > sent JoinGroup request contains the same subscription info as we
> >> used
> >> > to
> >> > > > assign the partitions previously (which should be likely the case
> in
> >> > > > practice). In addition, we should let static members to not send
> the
> >> > > > LeaveGroup request when it is gracefully shutdown, so that a
> static
> >> > > member
> >> > > > can only be leaving the group if its session has timed out, OR it
> >> has
> >> > > been
> >> > > > indicated to not exist in the group any more (details below).
> >> > > >
> >> > > > 1.b) We have a parallel discussion about Incremental Cooperative
> >> > > > Rebalancing, in which we will encode the "when to rebalance" logic
> >> at
> >> > the
> >> > > > application level, instead of at the protocol level. By doing this
> >> we
> >> > can
> >> > > > also enable a few other optimizations, e.g. at the Streams level
> to
> >> > first
> >> > > > build up the state store as standby tasks and then trigger a
> second
> >> > > > rebalance to actually migrate the active tasks while keeping the
> >> actual
> >> > > > rebalance latency and hence unavailability window to be small (
> >> > > >
> >> > > >
> >> > >
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=ZJlsB%2FHLhZykd9mtu5CINNNEqMBvX75bdhqR3IlxGI8%3D&amp;reserved=0
> >> > > ).
> >> > > > I'd propose we align
> >> > > > KIP-345 along with this idea, and hence do not add the
> >> > expansion_timeout
> >> > > as
> >> > > > part of the protocol layer, but only do that at the application's
> >> > > > coordinator / assignor layer (Connect, Streams, etc). We can
> still,
> >> > > > deprecate the "*group.initial.rebalance.delay.ms
> >> > > > <
> >> > > >
> >> > >
> >> >
> >>
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=DDhjV41nPU3euYCQ3w8WPENuw9fPB6ah2j6rF0JjRBg%3D&amp;reserved=0
> >> > > >*"
> >> > > > though as part of this KIP
> >> > > > since we have discussed about its limit and think it is actually
> >> not a
> >> > > very
> >> > > > good design and could be replaced with client-side logic above.
> >> > > >
> >> > > >
> >> > > > 2. I'd like to see your thoughts on the upgrade path for this KIP.
> >> More
> >> > > > specifically, let's say after we have upgraded broker version to
> be
> >> > able
> >> > > to
> >> > > > recognize the new versions of JoinGroup request and the admin
> >> requests,
> >> > > how
> >> > > > should we upgrade the clients and enable static groups? On top of
> my
> >> > head
> >> > > > if we do a rolling bounce in which we set the member.name config
> as
> >> > well
> >> > > > as
> >> > > > optionally increase the session.timeout config when we bounce each
> >> > > > instance, then during this rolling bounces we will have a group
> >> > contained
> >> > > > with both dynamic members and static members. It means that we
> >> should
> >> > > have
> >> > > > the group to allow such scenario (i.e. we cannot reject JoinGroup
> >> > > requests
> >> > > > from dynamic members), and hence the "member.name" -> "member.id"
> >> > > mapping
> >> > > > will only be partial at this scenario. Also could you describe if
> >> the
> >> > > > upgrade to the first version that support this feature would ever
> >> get
> >> > any
> >> > > > benefits, or only the future upgrade path for rolling bounces
> could
> >> get
> >> > > > benefits out of this feature?
> >> > > >
> >> > > > If that's the case and we will do 1) as suggested above, do we
> still
> >> > need
> >> > > > the enableStaticMembership and enableDynamicMembership admin
> >> requests
> >> > any
> >> > > > more? Seems it is not necessary any more as we will only have the
> >> > notion
> >> > > of
> >> > > > "dynamic or static members" that can co-exist in a group while
> >> there no
> >> > > > notion of "dynamic or static groups", and hence these two requests
> >> are
> >> > > not
> >> > > > needed anymore.
> >> > > >
> >> > > >
> >> > > > 3. We need to briefly talk about the implications for ACL as we
> >> > introduce
> >> > > > new admin requests that are related to a specific group.id. For
> >> > example,
> >> > > > we
> >> > > > need to make sure that whoever created the group or joined the
> group
> >> > can
> >> > > > actually send admin requests for the group, otherwise the
> >> application
> >> > > > owners need to bother the Kafka operators on a multi-tenant
> cluster
> >> > every
> >> > > > time they want to send any admin requests for their groups which
> >> would
> >> > be
> >> > > > an operational nightmare.
> >> > > >
> >> > > >
> >> > > > 4. I like Jason's suggestion of adding an optional field for the
> >> list
> >> > of
> >> > > > member names, and I'm wondering if that can be done as part of the
> >> > > > forceStaticRebalance request: i.e. by passing a list of members,
> we
> >> > will
> >> > > > enforce a rebalance immediately since it indicates that some
> static
> >> > > member
> >> > > > will be officially kicked out of the group and some new static
> >> members
> >> > > may
> >> > > > be added. So back to 1.a) above, a static member can only be
> kicked
> >> out
> >> > > of
> >> > > > the group if a) its session (arguably long period of time) has
> timed
> >> > out,
> >> > > > and b) this admin request explicitly state that it is no longer
> >> part of
> >> > > the
> >> > > > group. As for execution I'm fine with keeping it as a future work
> of
> >> > this
> >> > > > KIP if you'd like to make its scope smaller.
> >> > > >
> >> > > > Following are minor comments:
> >> > > >
> >> > > > 5. I'm not sure if we need to include "member.name" as part of
> the
> >> > > > OffsetCommitRequest for fencing purposes, as I think the memberId
> >> plus
> >> > > the
> >> > > > generation number should be sufficient for fencing even with
> static
> >> > > > members.
> >> > > >
> >> > > > 6. As mentioned above, if we agree to do 1) we can get rid of the
> "
> >> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> >> > > >
> >> > > >
> >> > > > Guozhang
> >> > > >
> >> > > >
> >> > > >
> >> > > >
> >> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com>
> >> wrote:
> >> > > >
> >> > > > > Hey Boyang,
> >> > > > >
> >> > > > > Thanks for the proposal! This is very useful. I have some
> comments
> >> > > below:
> >> > > > >
> >> > > > > 1) The motivation currently explicitly states that the goal is
> to
> >> > > improve
> >> > > > > performance for heavy state application. It seems that the
> >> motivation
> >> > > can
> >> > > > > be stronger with the following use-case. Currently for
> MirrorMaker
> >> > > > cluster
> >> > > > > with e.g. 100 MirrorMaker processes, it will take a long time to
> >> > > rolling
> >> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> >> > restart
> >> > > > > will trigger a rebalance which currently pause the consumption
> of
> >> the
> >> > > all
> >> > > > > partitions of the MirrorMaker cluster. With the change stated in
> >> this
> >> > > > > patch, as long as a MirrorMaker can restart within the specified
> >> > > timeout
> >> > > > > (e.g. 2 minutes), then we only need constant number of rebalance
> >> > (e.g.
> >> > > > for
> >> > > > > leader restart) for the entire rolling bounce, which will
> >> > significantly
> >> > > > > improves the availability of the MirrorMaker pipeline. In my
> >> opinion,
> >> > > the
> >> > > > > main benefit of the KIP is to avoid unnecessary rebalance if the
> >> > > consumer
> >> > > > > process can be restarted within soon, which helps performance
> >> even if
> >> > > > > overhead of state shuffling for a given process is small.
> >> > > > >
> >> > > > > 2) In order to simplify the KIP reading, can you follow the
> >> writeup
> >> > > style
> >> > > > > of other KIP (e.g. KIP-98) and list the interface change such as
> >> new
> >> > > > > configs (e.g. registration timeout), new request/response, new
> >> > > > AdminClient
> >> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> >> some
> >> > > of
> >> > > > > these are specified in the Proposed Change section which makes
> it
> >> a
> >> > bit
> >> > > > > inconvenient to understand the new interface that will be
> exposed
> >> to
> >> > > > user.
> >> > > > > Explanation of the current two-phase rebalance protocol probably
> >> can
> >> > be
> >> > > > > moved out of public interface section.
> >> > > > >
> >> > > > > 3) There are currently two version of JoinGroupRequest in the
> KIP
> >> and
> >> > > > only
> >> > > > > one of them has field memberId. This seems confusing.
> >> > > > >
> >> > > > > 4) It is mentioned in the KIP that "An admin API to force
> >> rebalance
> >> > > could
> >> > > > > be helpful here, but we will make a call once we finished the
> >> major
> >> > > > > implementation". So this seems to be still an open question in
> the
> >> > > > current
> >> > > > > design. We probably want to agree on this before voting for the
> >> KIP.
> >> > > > >
> >> > > > > 5) The KIP currently adds new config MEMBER_NAME for consumer.
> Can
> >> > you
> >> > > > > specify the name of the config key and the default config value?
> >> > > Possible
> >> > > > > default values include empty string or null (similar to
> >> > transaction.id
> >> > > > in
> >> > > > > producer config).
> >> > > > >
> >> > > > > 6) Regarding the use of the topic "static_member_map" to persist
> >> > member
> >> > > > > name map, currently if consumer coordinator broker goes offline,
> >> > > > rebalance
> >> > > > > is triggered and consumers will try connect to the new
> >> coordinator.
> >> > If
> >> > > > > these consumers can connect to the new coordinator within
> >> > > > > max.poll.interval.ms which by default is 5 minutes, given that
> >> > broker
> >> > > > can
> >> > > > > use a deterministic algorithm to determine the partition ->
> >> > member_name
> >> > > > > mapping, each consumer should get assigned the same set of
> >> partitions
> >> > > > > without requiring state shuffling. So it is not clear whether we
> >> > have a
> >> > > > > strong use-case for this new logic. Can you help clarify what is
> >> the
> >> > > > > benefit of using topic "static_member_map" to persist member
> name
> >> > map?
> >> > > > >
> >> > > > > 7) Regarding the introduction of the expensionTimeoutMs config,
> >> it is
> >> > > > > mentioned that "we are using expansion timeout to replace
> >> rebalance
> >> > > > > timeout, which is configured by max.poll.intervals from client
> >> side,
> >> > > and
> >> > > > > using registration timeout to replace session timeout".
> Currently
> >> the
> >> > > > > default max.poll.interval.ms is configured to be 5 minutes and
> >> there
> >> > > > will
> >> > > > > be only one rebalance if all new consumers can join within 5
> >> minutes.
> >> > > So
> >> > > > it
> >> > > > > is not clear whether we have a strong use-case for this new
> >> config.
> >> > Can
> >> > > > you
> >> > > > > explain what is the benefit of introducing this new config?
> >> > > > >
> >> > > > > 8) It is mentioned that "To distinguish between previous version
> >> of
> >> > > > > protocol, we will also increase the join group request version
> to
> >> v4
> >> > > when
> >> > > > > MEMBER_NAME is set" and "If the broker version is not the latest
> >> (<
> >> > > v4),
> >> > > > > the join group request shall be downgraded to v3 without setting
> >> the
> >> > > > member
> >> > > > > Id". It is probably simpler to just say that this feature is
> >> enabled
> >> > if
> >> > > > > JoinGroupRequest V4 is supported on both client and broker and
> >> > > > MEMBER_NAME
> >> > > > > is configured with non-empty string.
> >> > > > >
> >> > > > > 9) It is mentioned that broker may return
> >> NO_STATIC_MEMBER_INFO_SET
> >> > > error
> >> > > > > in OffsetCommitResponse for "commit requests under static
> >> > membership".
> >> > > > Can
> >> > > > > you clarify how broker determines whether the commit request is
> >> under
> >> > > > > static membership?
> >> > > > >
> >> > > > > Thanks,
> >> > > > > Dong
> >> > > > >
> >> > > >
> >> > > >
> >> > > > --
> >> > > > -- Guozhang
> >> > > >
> >> > >
> >> >
> >> >
> >> > --
> >> > -Regards,
> >> > Mayuresh R. Gharat
> >> > (862) 250-7125
> >> >
> >>
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Thanks a lot for replying to all the queries and discussions here, so
patiently.
Really appreciate it.

Had a few questions and suggestions after rereading the current version of
the KIP :


   1. Do you intend to have member.id is a static config like member.name
   after KIP-345 and KIP-394?
   2. Regarding "On client side, we add a new config called MEMBER_NAME in
   ConsumerConfig. On consumer service init, if the MEMBER_NAME config is set,
   we will put it in the initial join group request to identify itself as a
   static member (static membership); otherwise, we will still send
   UNKNOWN_MEMBER_ID to ask broker for allocating a new random ID (dynamic
   membership)."
      - What is the value of member_id sent in the first JoinGroupRequest
      when member_name is set (using static rebalance)? Is it UNKNOW_MEMBER_ID?
   3. Regarding "we are requiring member.id (if not unknown) to match the
   value stored in cache, otherwise reply MEMBER_ID_MISMATCH. The edge case
   that if we could have members with the same `member.name` (for example
   mis-configured instances with a valid member.id but added a used member
   name on runtime). When member name has duplicates, we could refuse join
   request from members with an outdated `member.id` (since we update the
   mapping upon each join group request). In an edge case where the client
   hits this exception in the response, it is suggesting that some other
   consumer takes its spot."
      - The part of "some other consumer takes the spot" would be
      intentional, right? Also when you say " The edge case that if we
      could have members with the same `member.name` (for example
      mis-configured instances *with a valid member.id <http://member.id> *but
      added a used member name on runtime).", what do you mean by *valid
      member id* here? Does it mean that there exist a mapping of
      member.name to member.id like *MemberA -> id1* on the
      GroupCoordinator and this consumer is trying to join with *member.name
      <http://member.name> = MemberB and member.id <http://member.id> = id1 *
      ?
   4. Depending on your explanation for point 2 and the point 3 above
   regarding returning back MEMBER_ID_MISMATCH on having a matching
   member_name but unknown member_id, if the consumer sends "UNKNOW_MEMBER_ID"
   on the first JoinGroupRequest and relies on the GroupCoordinator to give it
   a member_id, is the consumer suppose to remember member_id for
   joinGroupRequests? If yes, how are restarts handled?
   5. Regarding "So in summary, *the member will only be removed due to
   session timeout*. We shall remove it from both in-memory static member
   name mapping and member list."
      - If the rebalance is invoked manually using the the admin apis, how
      long should the group coordinator wait for the members of the
group to send
      a JoinGroupRequest for participating in the rebalance? How is a lagging
      consumer handled?
   6. Another detail to take care is that we need to automatically take the
   hash of group id so that we know which broker to send this request to.
      - I assume this should be same as the way we find the coordinator,
      today right? If yes, should we specify it in the KIP ?
   7. Are there any specific failure scenarios when you say "other
   potential failure cases."? It would be good to mention them explicitly, if
   you think there are any.
   8. It would be good to have a rollback plan as you have for roll forward
   in the KIP.

Thanks,

Mayuresh

On Mon, Nov 26, 2018 at 8:17 AM Mayuresh Gharat <gh...@gmail.com>
wrote:

> Hi Boyang,
>
> Do you have a discuss thread for KIP-394 that you mentioned here ?
>
> Thanks,
>
> Mayuresh
>
> On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com> wrote:
>
>> Hey Dong, thanks for the follow-up here!
>>
>>
>> 1) It is not very clear to the user what is the difference between
>> member.name and client.id as both seems to be used to identify the
>> consumer. I am wondering if it would be more intuitive to name it
>> group.member.name (preferred choice since it matches the current group.id
>> config name) or rebalance.member.name to explicitly show that the id is
>> solely used for rebalance.
>> Great question. I feel `member.name` is enough to explain itself, it
>> seems not very
>> helpful to make the config name longer. Comparing `name` with `id` gives
>> user the
>> impression that they have the control over it with customized rule than
>> library decided.
>>
>> 2) In the interface change section it is said that
>> GroupMaxSessionTimeoutMs
>> will be changed to 30 minutes. It seems to suggest that we will change the
>> default value of this config. It does not seem necessary to increase the
>> time of consumer failure detection when user doesn't use static
>> membership.
>> Also, say static membership is enabled, then this default config change
>> will cause a partition to be unavailable for consumption for 30 minutes if
>> there is hard consumer failure, which seems to be worse experience than
>> having unnecessary rebalance (when this timeout is small), particularly
>> for
>> new users of Kafka. Could you explain more why we should make this change?
>> We are not changing the default session timeout value. We are just
>> changing the
>> cap we are enforcing on the session timeout max value. So this change is
>> not affecting
>> what kind of membership end user is using, and loosing the cap is giving
>> end user
>> more flexibility on trade-off between liveness and stability.
>>
>> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
>> into one error? It seems that these two errors are currently handled by
>> the
>> consumer in the same way. And we don't also don't expect
>> MEMBER_ID_MISMATCH
>> to happen. Thus it is not clear what is the benefit of having two errors.
>> I agree that we should remove DUPLICATE_STATIC_MEMBER error because with
>> the KIP-394<
>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member+id+for+initial+join+group+request
>> >
>> we will automatically fence all join requests with UNKNOWN_MEMBER_ID.
>>
>> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
>> member name which is already in the consumer group, however the member id
>> was missing". After a consumer is restarted, it will send a
>> JoinGroupRequest with an existing memberName (as the coordinator has not
>> expired this member from the memory) and memberId
>> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
>> across consumer restart in the consumer side). Does it mean that
>> JoinGroupRequest from a newly restarted consumer will always be rejected
>> until the sessionTimeoutMs has passed?
>> Same answer as question 3). This part of the logic shall be removed from
>> the proposal.
>>
>> 5) It seems that we always add two methods to the interface
>> org.apache.kafka.clients.admin.AdminClient.java, one with options and the
>> other without option. Could this be specified in the interface change
>> section?
>> Sounds good! Added both methods.
>>
>> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
>> rebalance? If so, we probably want to specify the command line tool
>> interface similar to
>>
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
>> .
>> Added the script.
>>
>> 7) Would it be simpler to replace name "forceStaticRebalance" with
>> "invokeConsumerRebalance"? It is not very clear what is the extra meaning
>> of world "force" as compared to "trigger" or "invoke". And it seems
>> simpler
>> to allows this API to trigger rebalance regardless of whether consumer is
>> configured with memberName.
>> Sounds good. Right now I feel for both static and dynamic membership it is
>> more manageable to introduce the consumer rebalance method through admin
>> client API.
>>
>> 8) It is not very clear how the newly added AdminClient API trigger
>> rebalance. For example, does it send request? Can this be explained in the
>> KIP?
>>
>> Sure, I will add more details to the API.
>>
>>
>> Thanks again for the helpful suggestions!
>>
>>
>> Best,
>> Boyang
>>
>> ________________________________
>> From: Dong Lin <li...@gmail.com>
>> Sent: Saturday, November 24, 2018 2:54 PM
>> To: dev
>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
>> specifying member id
>>
>> Hey Boyang,
>>
>> Thanks for the update! Here are some followup comments:
>>
>> 1) It is not very clear to the user what is the difference between
>> member.name and client.id as both seems to be used to identify the
>> consumer. I am wondering if it would be more intuitive to name it
>> group.member.name (preferred choice since it matches the current group.id
>> config name) or rebalance.member.name to explicitly show that the id is
>> solely used for rebalance.
>>
>> 2) In the interface change section it is said that
>> GroupMaxSessionTimeoutMs
>> will be changed to 30 minutes. It seems to suggest that we will change the
>> default value of this config. It does not seem necessary to increase the
>> time of consumer failure detection when user doesn't use static
>> membership.
>> Also, say static membership is enabled, then this default config change
>> will cause a partition to be unavailable for consumption for 30 minutes if
>> there is hard consumer failure, which seems to be worse experience than
>> having unnecessary rebalance (when this timeout is small), particularly
>> for
>> new users of Kafka. Could you explain more why we should make this change?
>>
>> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
>> into one error? It seems that these two errors are currently handled by
>> the
>> consumer in the same way. And we don't also don't expect
>> MEMBER_ID_MISMATCH
>> to happen. Thus it is not clear what is the benefit of having two errors.
>>
>> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
>> member name which is already in the consumer group, however the member id
>> was missing". After a consumer is restarted, it will send a
>> JoinGroupRequest with an existing memberName (as the coordinator has not
>> expired this member from the memory) and memberId
>> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
>> across consumer restart in the consumer side). Does it mean that
>> JoinGroupRequest from a newly restarted consumer will always be rejected
>> until the sessionTimeoutMs has passed?
>>
>> 5) It seems that we always add two methods to the interface
>> org.apache.kafka.clients.admin.AdminClient.java, one with options and the
>> other without option. Could this be specified in the interface change
>> section?
>>
>> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
>> rebalance? If so, we probably want to specify the command line tool
>> interface similar to
>>
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
>> .
>>
>> 7) Would it be simpler to replace name "forceStaticRebalance" with
>> "invokeConsumerRebalance"? It is not very clear what is the extra meaning
>> of world "force" as compared to "trigger" or "invoke". And it seems
>> simpler
>> to allows this API to trigger rebalance regardless of whether consumer is
>> configured with memberName.
>>
>> 8) It is not very clear how the newly added AdminClient API trigger
>> rebalance. For example, does it send request? Can this be explained in the
>> KIP?
>>
>> Thanks,
>> Dong
>>
>>
>> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com> wrote:
>>
>> > Hey Mayuresh,
>> >
>> >
>> > thanks for your feedbacks! I will try do another checklist here.
>> >
>> >
>> > > By this you mean, even if the application has not called
>> > > KafkaConsumer.poll() within session timeout, it will not be sending
>> the
>> > > LeaveGroup request, right?
>> >
>> > Yep it's true, we will prevent client from sending leave group request
>> > when they are set with `member.name`.
>> >
>> >
>> > > When is the member.name removed from this map?
>> > Good question, we will only kick off member due to session timeout
>> within
>> > static membership. Let me update the KIP to clearly assert that.
>> >
>> > > How is this case (missing member id) handled on the client side? What
>> is
>> > the application that
>> > > is using the KafkaConsumer suppose to do in this scenario?
>> > I have extended the two exceptions within join group response V4.
>> > Basically I define both corresponding actions to be immediate failing
>> > client application, because so far it is unknown what kind of client
>> issue
>> > could trigger them. After the first version, we will keep enhance the
>> error
>> > handling logic!
>> >
>> > > This would mean that it might take more time to detect unowned topic
>> > > partitions and may cause delay for applications that perform data
>> > mirroring
>> > > tasks. I discussed this with our sre and we have a suggestion to make
>> > here
>> > > as listed below separately.
>> > The goal of extending session timeout cap is for users with good client
>> > side monitoring tools that could auto-heal the dead consumers very
>> fast. So
>> > it is optional (and personal) to extend session timeout to a reasonable
>> > number with different client scenarios.
>> >
>> > > you meant remove unjoined members of the group, right ?
>> > Yep, there is a typo. Thanks for catching this!
>> >
>> > > What do you mean by " Internally we would optimize this logic by
>> having
>> > > rebalance timeout only in charge of stopping prepare rebalance stage,
>> > > without removing non-responsive members immediately." There would not
>> be
>> > a
>> > > full rebalance if the lagging consumer sent a JoinGroup request later,
>> > > right ? If yes, can you highlight this in the KIP ?
>> > No, there won't be. We want to limit the rebalance timeout functionality
>> > to only use as a timer to
>> > end prepare rebalance stage. This way, late joining static members will
>> > not trigger further rebalance
>> > as long as they are within session timeout. I added your highlight to
>> the
>> > KIP!
>> >
>> > > The KIP talks about scale up scenario but its not quite clear how we
>> > > handle it. Are we adding a separate "expansion.timeout" or we adding
>> > status
>> > > "learner" ?. Can you shed more light on how this is handled in the
>> KIP,
>> > if
>> > > its handled?
>> > Updated the KIP: we shall not cover scale up case in 345, because we
>> > believe client side could
>> > better handle this logic.
>> >
>> > > I think Jason had brought this up earlier about having a way to say
>> how
>> > > many members/consumer hosts are you choosing to be in the consumer
>> group.
>> > > If we can do this, then in case of mirroring applications we can do
>> this
>> > :
>> > > Lets say we have a mirroring application that consumes from Kafka
>> cluster
>> > > A and produces to Kafka cluster B.
>> > > Depending on the data and the Kafka cluster configuration, Kafka
>> service
>> > > providers can set a mirroring group saying that it will take, for
>> example
>> > > 300 consumer hosts/members to achieve the desired throughput and
>> latency
>> > > for mirroring and can have additional 10 consumer hosts as spare in
>> the
>> > > same group.
>> > > So when the first 300 members/consumers to join the group will start
>> > > mirroring the data from Kafka cluster A to Kafka cluster B.
>> > > The remaining 10 consumer members can sit idle.
>> > > The moment one of the consumer (for example: consumer number 54) from
>> the
>> > > first 300 members go out of the group (crossed session timeout), it
>> (the
>> > > groupCoordinator) can just assign the topicPartitions from the
>> consumer
>> > > member 54 to one of the spare hosts.
>> > > Once the consumer member 54 comes back up, it can start as being a
>> part
>> > of
>> > > the spare pool.
>> > > This enables us to have lower session timeouts and low latency
>> mirroring,
>> > > in cases where the service providers are OK with having spare hosts.
>> > > This would mean that we would tolerate n consumer members leaving and
>> > > rejoining the group and still provide low latency as long as n <=
>> number
>> > of
>> > > spare consumers.
>> > > If there are no spare host available, we can get back to the idea as
>> > > described in the KIP.
>> > Great idea! In fact on top of static membership we could later introduce
>> > APIs to set hard-coded
>> > client ids to the group and replace the dead host, or as you proposed to
>> > define spare host as
>> > what I understood as hot backup. I will put both Jason and your
>> > suggestions into a separate section
>> > called "Future works". Note that this spare host idea may be also
>> solvable
>> > through rebalance protocol
>> > IMO.
>> >
>> > Thank you again for the great feedback!
>> >
>> > Boyang
>> > ________________________________
>> > From: Boyang Chen <bc...@outlook.com>
>> > Sent: Thursday, November 22, 2018 3:39 PM
>> > To: dev@kafka.apache.org
>> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
>> > specifying member id
>> >
>> > Hey Dong, sorry for missing your message. I couldn't find your email on
>> my
>> > thread, so I will just do a checklist here!
>> >
>> >
>> > 1) The motivation currently explicitly states that the goal is to
>> improve
>> >
>> > performance for heavy state application. It seems that the motivation
>> can
>> >
>> > be stronger with the following use-case. Currently for MirrorMaker
>> cluster
>> >
>> > with e.g. 100 MirrorMaker processes, it will take a long time to rolling
>> >
>> > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
>> >
>> > will trigger a rebalance which currently pause the consumption of the
>> all
>> >
>> > partitions of the MirrorMaker cluster. With the change stated in this
>> >
>> > patch, as long as a MirrorMaker can restart within the specified timeout
>> >
>> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
>> for
>> >
>> > leader restart) for the entire rolling bounce, which will significantly
>> >
>> > improves the availability of the MirrorMaker pipeline. In my opinion,
>> the
>> >
>> > main benefit of the KIP is to avoid unnecessary rebalance if the
>> consumer
>> >
>> > process can be restarted within soon, which helps performance even if
>> >
>> > overhead of state shuffling for a given process is small.
>> >
>> > I just rephrased this part and added it to the KIP. Thanks for making
>> the
>> > motivation more solid!
>> >
>> > 2) In order to simplify the KIP reading, can you follow the writeup
>> style
>> > of other KIP (e.g. KIP-98) and list the interface change such as new
>> > configs (e.g. registration timeout), new request/response, new
>> AdminClient
>> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
>> > these are specified in the Proposed Change section which makes it a bit
>> > inconvenient to understand the new interface that will be exposed to
>> user.
>> > Explanation of the current two-phase rebalance protocol probably can be
>> > moved out of public interface section.
>> > This is a great suggestion! I just consolidated all the public API
>> > changes, and the whole KIP
>> > looks much more organized!
>> >
>> > 3) There are currently two version of JoinGroupRequest in the KIP and
>> only
>> > one of them has field memberId. This seems confusing.
>> > Yep, I already found this issue and fixed it.
>> >
>> > 4) It is mentioned in the KIP that "An admin API to force rebalance
>> could
>> > be helpful here, but we will make a call once we finished the major
>> > implementation". So this seems to be still an open question in the
>> current
>> > design. We probably want to agree on this before voting for the KIP.
>> > We have finalized the idea that this API is needed.
>> >
>> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
>> > specify the name of the config key and the default config value?
>> Possible
>> > default values include empty string or null (similar to transaction.id<
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=kkCtxHpsAZpbJZpUc52dtv6ac8UJOx6CQlts3CPjDh8%3D&amp;reserved=0
>> >
>> > in
>> > producer config).
>> > I have defined the `member.name` in "New configuration" section.
>> >
>> > 6) Regarding the use of the topic "static_member_map" to persist member
>> > name map, currently if consumer coordinator broker goes offline,
>> rebalance
>> > is triggered and consumers will try connect to the new coordinator. If
>> > these consumers can connect to the new coordinator within
>> > max.poll.interval.ms<
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
>> >
>> > which by default is 5 minutes, given that broker can
>> > use a deterministic algorithm to determine the partition -> member_name
>> > mapping, each consumer should get assigned the same set of partitions
>> > without requiring state shuffling. So it is not clear whether we have a
>> > strong use-case for this new logic. Can you help clarify what is the
>> > benefit of using topic "static_member_map" to persist member name map?
>> > I have discussed with Guozhang offline, and I believe reusing the
>> current
>> > `_consumer_offsets`
>> > topic is a better and unified solution.
>> >
>> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
>> > mentioned that "we are using expansion timeout to replace rebalance
>> > timeout, which is configured by max.poll.intervals from client side, and
>> > using registration timeout to replace session timeout". Currently the
>> > default max.poll.interval.ms<
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
>> >
>> > is configured to be 5 minutes and there will
>> > be only one rebalance if all new consumers can join within 5 minutes.
>> So it
>> > is not clear whether we have a strong use-case for this new config. Can
>> you
>> > explain what is the benefit of introducing this new config?
>> > Previously our goal is to use expansion timeout as a workaround for
>> > triggering multiple
>> > rebalances when scaling up members are not joining at the same time. It
>> is
>> > decided to
>> > be addressed by client side protocol change, so we will not introduce
>> > expansion timeout.
>> >
>> > 8) It is mentioned that "To distinguish between previous version of
>> > protocol, we will also increase the join group request version to v4
>> when
>> > MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
>> > the join group request shall be downgraded to v3 without setting the
>> member
>> > Id". It is probably simpler to just say that this feature is enabled if
>> > JoinGroupRequest V4 is supported on both client and broker and
>> MEMBER_NAME
>> > is configured with non-empty string.
>> > Yep, addressed this!
>> >
>> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
>> error
>> > in OffsetCommitResponse for "commit requests under static membership".
>> Can
>> > you clarify how broker determines whether the commit request is under
>> > static membership?
>> >
>> > We have agreed that commit request shouldn't be affected by the new
>> > membership, thus
>> > removing it here. Thanks for catching this!
>> >
>> > Let me know if you have further suggestions or concerns. Thank you for
>> > your valuable feedback
>> > to help me design the KIP better! (And I will try to address your
>> > feedbacks in next round Mayuresh ??)
>> >
>> > Best,
>> > Boyang
>> > ________________________________
>> > From: Mayuresh Gharat <gh...@gmail.com>
>> > Sent: Wednesday, November 21, 2018 7:50 AM
>> > To: dev@kafka.apache.org
>> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
>> > specifying member id
>> >
>> > Hi Boyang,
>> >
>> > Thanks for updating the KIP. This is a step good direction for stateful
>> > applications and also mirroring applications whose latency is affected
>> due
>> > to the rebalance issues that we have today.
>> >
>> > I had a few questions on the current version of the KIP :
>> > For the effectiveness of the KIP, consumer with member.name set will
>> *not
>> > send leave group request* when they go offline
>> >
>> > > By this you mean, even if the application has not called
>> > > KafkaConsumer.poll() within session timeout, it will not be sending
>> the
>> > > LeaveGroup request, right?
>> > >
>> >
>> > Broker will maintain an in-memory mapping of {member.name ? member.id}
>> to
>> > track member uniqueness.
>> >
>> > > When is the member.name removed from this map?
>> > >
>> >
>> > Member.id must be set if the *member.name <
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=aSTzJlF4GPFGKhi5K7MPZozMn67718GWpqEYxFay%2BZs%3D&amp;reserved=0
>> >
>> > *is already
>> > within the map. Otherwise reply MISSING_MEMBER_ID
>> >
>> > > How is this case handled on the client side? What is the application
>> that
>> > > is using the KafkaConsumer suppose to do in this scenario?
>> > >
>> >
>> > Session timeout is the timeout we will trigger rebalance when a member
>> goes
>> > offline for too long (not sending heartbeat request). To make static
>> > membership effective, we should increase the default max session
>> timeout to
>> > 30 min so that end user could config it freely.
>> >
>> > > This would mean that it might take more time to detect unowned topic
>> > > partitions and may cause delay for applications that perform data
>> > mirroring
>> > > tasks. I discussed this with our sre and we have a suggestion to make
>> > here
>> > > as listed below separately.
>> > >
>> >
>> > Currently there is a config called *rebalance timeout* which is
>> configured
>> > by consumer *max.poll.intervals*. The reason we set it to poll interval
>> is
>> > because consumer could only send request within the call of poll() and
>> we
>> > want to wait sufficient time for the join group request. When reaching
>> > rebalance timeout, the group will move towards completingRebalance stage
>> > and remove unjoined groups
>> >
>> > > you meant remove unjoined members of the group, right ?
>> > >
>> >
>> > Currently there is a config called *rebalance timeout* which is
>> configured
>> > by consumer *max.poll.intervals*. The reason we set it to poll interval
>> is
>> > because consumer could only send request within the call of poll() and
>> we
>> > want to wait sufficient time for the join group request. When reaching
>> > rebalance timeout, the group will move towards completingRebalance stage
>> > and remove unjoined groups. This is actually conflicting with the
>> design of
>> > static membership, because those temporarily unavailable members will
>> > potentially reattempt the join group and trigger extra rebalances.
>> > Internally we would optimize this logic by having rebalance timeout
>> only in
>> > charge of stopping prepare rebalance stage, without removing
>> non-responsive
>> > members immediately.
>> >
>> > > What do you mean by " Internally we would optimize this logic by
>> having
>> > > rebalance timeout only in charge of stopping prepare rebalance stage,
>> > > without removing non-responsive members immediately." There would not
>> be
>> > a
>> > > full rebalance if the lagging consumer sent a JoinGroup request later,
>> > > right ? If yes, can you highlight this in the KIP ?
>> > >
>> >
>> > Scale Up
>> >
>> > > The KIP talks about scale up scenario but its not quite clear how we
>> > > handle it. Are we adding a separate "expansion.timeout" or we adding
>> > status
>> > > "learner" ?. Can you shed more light on how this is handled in the
>> KIP,
>> > if
>> > > its handled?
>> > >
>> >
>> >
>> > *Discussion*
>> > Larger session timeouts causing latency rise for getting data for
>> un-owned
>> > topic partitions :
>> >
>> > > I think Jason had brought this up earlier about having a way to say
>> how
>> > > many members/consumer hosts are you choosing to be in the consumer
>> group.
>> > > If we can do this, then in case of mirroring applications we can do
>> this
>> > :
>> > > Lets say we have a mirroring application that consumes from Kafka
>> cluster
>> > > A and produces to Kafka cluster B.
>> > > Depending on the data and the Kafka cluster configuration, Kafka
>> service
>> > > providers can set a mirroring group saying that it will take, for
>> example
>> > > 300 consumer hosts/members to achieve the desired throughput and
>> latency
>> > > for mirroring and can have additional 10 consumer hosts as spare in
>> the
>> > > same group.
>> > > So when the first 300 members/consumers to join the group will start
>> > > mirroring the data from Kafka cluster A to Kafka cluster B.
>> > > The remaining 10 consumer members can sit idle.
>> > > The moment one of the consumer (for example: consumer number 54) from
>> the
>> > > first 300 members go out of the group (crossed session timeout), it
>> (the
>> > > groupCoordinator) can just assign the topicPartitions from the
>> consumer
>> > > member 54 to one of the spare hosts.
>> > > Once the consumer member 54 comes back up, it can start as being a
>> part
>> > of
>> > > the spare pool.
>> > > This enables us to have lower session timeouts and low latency
>> mirroring,
>> > > in cases where the service providers are OK with having spare hosts.
>> > > This would mean that we would tolerate n consumer members leaving and
>> > > rejoining the group and still provide low latency as long as n <=
>> number
>> > of
>> > > spare consumers.
>> > > If there are no spare host available, we can get back to the idea as
>> > > described in the KIP.
>> > >
>> >
>> > Thanks,
>> >
>> > Mayuresh
>> >
>> >
>> >
>> >
>> >
>> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
>> > konstantine@confluent.io> wrote:
>> >
>> > > Hi Boyang.
>> > >
>> > > Thanks for preparing this KIP! It is making good progress and will be
>> a
>> > > great improvement for stateful Kafka applications.
>> > >
>> > > Apologies for my late reply, I was away for a while. Lots of great
>> > comments
>> > > so far, so I'll probably second most of them in what I suggest below
>> at
>> > > this point.
>> > >
>> > > When I first read the KIP, I wanted to start at the end with something
>> > that
>> > > wasn't highlighted a lot. That was the topic related to handling
>> > duplicate
>> > > members. I see now that the initial suggestion of handling this
>> situation
>> > > during offset commit has been removed, and I agree with that. Issues
>> > > related to membership seem to be handled better when the member joins
>> the
>> > > group rather than when it tries to commit offsets. This also
>> simplifies
>> > how
>> > > many request types need to change in order to incorporate the new
>> member
>> > > name field.
>> > >
>> > > I also agree with what Jason and Guozhang have said regarding
>> timeouts.
>> > > Although semantically, it's easier to think of every operation having
>> its
>> > > own timeout, operationally this can become a burden. Thus,
>> consolidation
>> > > seems preferable here. The definition of embedded protocols on top of
>> the
>> > > base group membership protocol for rebalancing gives enough
>> flexibility
>> > to
>> > > address such needs in each client component separately.
>> > >
>> > > Finally, some minor comments:
>> > > In a few places the new/proposed changes are referred to as "current".
>> > > Which is a bit confusing considering that there is a protocol in place
>> > > already, and by "current" someone might understand the existing one.
>> I'd
>> > > recommend using new/proposed or equivalent when referring to changes
>> > > introduced with KIP-345 and current/existing or equivalent when
>> referring
>> > > to existing behavior.
>> > >
>> > > There's the following sentence in the "Public Interfaces" section:
>> > > "Since for many stateful consumer/stream applications, the state
>> > shuffling
>> > > is more painful than short time partial unavailability."
>> > > However, my understanding is that the changes proposed with KIP-345
>> will
>> > > not exploit any partial availability. A suggestion for dealing with
>> > > temporary imbalances has been made in "Incremental Cooperative
>> > Rebalancing"
>> > > which can work well with KIP-345, but here I don't see proposed
>> changes
>> > > that suggest that some resources (e.g. partitions) will keep being
>> used
>> > > while others will not be utilized. Thus, you might want to adjust this
>> > > sentence. Correct me if I'm missing something related to that.
>> > >
>> > > In the rejected alternatives, under point 2) I read "we can copy the
>> > member
>> > > id to the config files". I believe it means to say "member name"
>> unless
>> > I'm
>> > > missing something about reusing member ids. Also below I read: "By
>> > allowing
>> > > consumers to optionally specifying a member id" which probably implies
>> > > "member name" again. In a sense this section highlights a potential
>> > > confusion between member name and member id. I wonder if we could
>> come up
>> > > with a better term for the new field. StaticTag, StaticLabel, or even
>> > > StaticName are some suggestions that could potentially help with
>> > confusion
>> > > between MemberId and MemberName and what corresponds to what. But I
>> > > wouldn't like to disrupt the discussion with naming conventions too
>> much
>> > at
>> > > this point. I just mention it here as a thought.
>> > >
>> > > Looking forward to see the final details of this KIP. Great work so
>> far!
>> > >
>> > > Konstantine
>> > >
>> > >
>> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com>
>> wrote:
>> > >
>> > > > Thanks Guozhang for the great summary here, and I have been
>> following
>> > up
>> > > > the action items here.
>> > > >
>> > > >
>> > > >   1.  I already updated the KIP to remove the expansion timeout and
>> > > > registration timeout. Great to see them being addressed in client
>> side!
>> > > >   2.  I double checked the design and I believe that it is ok to
>> have
>> > > both
>> > > > static member and dynamic member co-exist in the same group. So the
>> > > upgrade
>> > > > shouldn't be destructive and we are removing the two membership
>> > protocol
>> > > > switching APIs.
>> > > >   3.  I only have question about this one. I'm still reading the
>> > > KafkaApis
>> > > > code here. Should I just use the same authorization logic for
>> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
>> > > >   4.  I'm very excited to see this work with K8! Like you suggested,
>> > this
>> > > > feature could be better addressed in a separate KIP because it is
>> > pretty
>> > > > independent. I could start drafting the KIP once the current
>> proposal
>> > is
>> > > > approved.
>> > > >   5.  I believe that we don't need fencing in offset commit request,
>> > > since
>> > > > duplicate member.name issue could be handled by join group
>> request. We
>> > > > shall reject join group with known member name but no member id
>> (which
>> > > > means we already have an active member using this identity).
>> > > >   6.  I agree to remove that internal config once we move forward
>> with
>> > > > static membership. And I already removed the entire section from the
>> > KIP.
>> > > >
>> > > > Let me know if you have other concerns.
>> > > >
>> > > > Best,
>> > > > Boyang
>> > > > ________________________________
>> > > > From: Guozhang Wang <wa...@gmail.com>
>> > > > Sent: Tuesday, November 20, 2018 4:21 PM
>> > > > To: dev
>> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
>> by
>> > > > specifying member id
>> > > >
>> > > > Hello Boyang,
>> > > >
>> > > > Thanks a lot for the KIP! It is a great write-up and I appreciate
>> your
>> > > > patience answering to the feedbacks from the community. I'd like to
>> add
>> > > my
>> > > > 2cents here:
>> > > >
>> > > > 1. By introducing another two timeout configs, registration_timeout
>> and
>> > > > expansion_timeout, we are effectively having four timeout configs:
>> > > session
>> > > > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
>> > > client
>> > > > side), and these two. Interplaying these timeout configs can be
>> quite
>> > > hard
>> > > > for users with such complexity, and hence I'm wondering if we can
>> > > simplify
>> > > > the situation with as less possible timeout configs as possible.
>> Here
>> > is
>> > > a
>> > > > concrete suggestion I'd like propose:
>> > > >
>> > > > 1.a) Instead of introducing a registration_timeout in addition to
>> the
>> > > > session_timeout for static members, we can just reuse the
>> > session_timeout
>> > > > and ask users to set it to a larger value when they are upgrading a
>> > > dynamic
>> > > > client to a static client by setting the "member.name" at the same
>> > time.
>> > > > By
>> > > > default, the broker-side min.session.timeout is 6 seconds and
>> > > > max.session.timeout is 5 minutes, which seems reasonable to me (we
>> can
>> > of
>> > > > course modify this broker config to enlarge the valid interval if we
>> > want
>> > > > in practice). And then we should also consider removing the
>> condition
>> > for
>> > > > marking a client as failed if the rebalance timeout has reached
>> while
>> > the
>> > > > JoinGroup was not received, so that the semantics of session_timeout
>> > and
>> > > > rebalance_timeout are totally separated: the former is only used to
>> > > > determine if a consumer member of the group should be marked as
>> failed
>> > > and
>> > > > kicked out of the group, and the latter is only used to determine
>> the
>> > > > longest time coordinator should wait for PREPARE_REBALANCE phase. In
>> > > other
>> > > > words if a member did not send the JoinGroup in time of the
>> > > > rebalance_timeout, we still include it in the new generation of the
>> > group
>> > > > and use its old subscription info to send to leader for assignment.
>> > Later
>> > > > if the member came back with HeartBeat request, we can still follow
>> the
>> > > > normal path to bring it to the latest generation while checking that
>> > its
>> > > > sent JoinGroup request contains the same subscription info as we
>> used
>> > to
>> > > > assign the partitions previously (which should be likely the case in
>> > > > practice). In addition, we should let static members to not send the
>> > > > LeaveGroup request when it is gracefully shutdown, so that a static
>> > > member
>> > > > can only be leaving the group if its session has timed out, OR it
>> has
>> > > been
>> > > > indicated to not exist in the group any more (details below).
>> > > >
>> > > > 1.b) We have a parallel discussion about Incremental Cooperative
>> > > > Rebalancing, in which we will encode the "when to rebalance" logic
>> at
>> > the
>> > > > application level, instead of at the protocol level. By doing this
>> we
>> > can
>> > > > also enable a few other optimizations, e.g. at the Streams level to
>> > first
>> > > > build up the state store as standby tasks and then trigger a second
>> > > > rebalance to actually migrate the active tasks while keeping the
>> actual
>> > > > rebalance latency and hence unavailability window to be small (
>> > > >
>> > > >
>> > >
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=ZJlsB%2FHLhZykd9mtu5CINNNEqMBvX75bdhqR3IlxGI8%3D&amp;reserved=0
>> > > ).
>> > > > I'd propose we align
>> > > > KIP-345 along with this idea, and hence do not add the
>> > expansion_timeout
>> > > as
>> > > > part of the protocol layer, but only do that at the application's
>> > > > coordinator / assignor layer (Connect, Streams, etc). We can still,
>> > > > deprecate the "*group.initial.rebalance.delay.ms
>> > > > <
>> > > >
>> > >
>> >
>> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=DDhjV41nPU3euYCQ3w8WPENuw9fPB6ah2j6rF0JjRBg%3D&amp;reserved=0
>> > > >*"
>> > > > though as part of this KIP
>> > > > since we have discussed about its limit and think it is actually
>> not a
>> > > very
>> > > > good design and could be replaced with client-side logic above.
>> > > >
>> > > >
>> > > > 2. I'd like to see your thoughts on the upgrade path for this KIP.
>> More
>> > > > specifically, let's say after we have upgraded broker version to be
>> > able
>> > > to
>> > > > recognize the new versions of JoinGroup request and the admin
>> requests,
>> > > how
>> > > > should we upgrade the clients and enable static groups? On top of my
>> > head
>> > > > if we do a rolling bounce in which we set the member.name config as
>> > well
>> > > > as
>> > > > optionally increase the session.timeout config when we bounce each
>> > > > instance, then during this rolling bounces we will have a group
>> > contained
>> > > > with both dynamic members and static members. It means that we
>> should
>> > > have
>> > > > the group to allow such scenario (i.e. we cannot reject JoinGroup
>> > > requests
>> > > > from dynamic members), and hence the "member.name" -> "member.id"
>> > > mapping
>> > > > will only be partial at this scenario. Also could you describe if
>> the
>> > > > upgrade to the first version that support this feature would ever
>> get
>> > any
>> > > > benefits, or only the future upgrade path for rolling bounces could
>> get
>> > > > benefits out of this feature?
>> > > >
>> > > > If that's the case and we will do 1) as suggested above, do we still
>> > need
>> > > > the enableStaticMembership and enableDynamicMembership admin
>> requests
>> > any
>> > > > more? Seems it is not necessary any more as we will only have the
>> > notion
>> > > of
>> > > > "dynamic or static members" that can co-exist in a group while
>> there no
>> > > > notion of "dynamic or static groups", and hence these two requests
>> are
>> > > not
>> > > > needed anymore.
>> > > >
>> > > >
>> > > > 3. We need to briefly talk about the implications for ACL as we
>> > introduce
>> > > > new admin requests that are related to a specific group.id. For
>> > example,
>> > > > we
>> > > > need to make sure that whoever created the group or joined the group
>> > can
>> > > > actually send admin requests for the group, otherwise the
>> application
>> > > > owners need to bother the Kafka operators on a multi-tenant cluster
>> > every
>> > > > time they want to send any admin requests for their groups which
>> would
>> > be
>> > > > an operational nightmare.
>> > > >
>> > > >
>> > > > 4. I like Jason's suggestion of adding an optional field for the
>> list
>> > of
>> > > > member names, and I'm wondering if that can be done as part of the
>> > > > forceStaticRebalance request: i.e. by passing a list of members, we
>> > will
>> > > > enforce a rebalance immediately since it indicates that some static
>> > > member
>> > > > will be officially kicked out of the group and some new static
>> members
>> > > may
>> > > > be added. So back to 1.a) above, a static member can only be kicked
>> out
>> > > of
>> > > > the group if a) its session (arguably long period of time) has timed
>> > out,
>> > > > and b) this admin request explicitly state that it is no longer
>> part of
>> > > the
>> > > > group. As for execution I'm fine with keeping it as a future work of
>> > this
>> > > > KIP if you'd like to make its scope smaller.
>> > > >
>> > > > Following are minor comments:
>> > > >
>> > > > 5. I'm not sure if we need to include "member.name" as part of the
>> > > > OffsetCommitRequest for fencing purposes, as I think the memberId
>> plus
>> > > the
>> > > > generation number should be sufficient for fencing even with static
>> > > > members.
>> > > >
>> > > > 6. As mentioned above, if we agree to do 1) we can get rid of the "
>> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
>> > > >
>> > > >
>> > > > Guozhang
>> > > >
>> > > >
>> > > >
>> > > >
>> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com>
>> wrote:
>> > > >
>> > > > > Hey Boyang,
>> > > > >
>> > > > > Thanks for the proposal! This is very useful. I have some comments
>> > > below:
>> > > > >
>> > > > > 1) The motivation currently explicitly states that the goal is to
>> > > improve
>> > > > > performance for heavy state application. It seems that the
>> motivation
>> > > can
>> > > > > be stronger with the following use-case. Currently for MirrorMaker
>> > > > cluster
>> > > > > with e.g. 100 MirrorMaker processes, it will take a long time to
>> > > rolling
>> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
>> > restart
>> > > > > will trigger a rebalance which currently pause the consumption of
>> the
>> > > all
>> > > > > partitions of the MirrorMaker cluster. With the change stated in
>> this
>> > > > > patch, as long as a MirrorMaker can restart within the specified
>> > > timeout
>> > > > > (e.g. 2 minutes), then we only need constant number of rebalance
>> > (e.g.
>> > > > for
>> > > > > leader restart) for the entire rolling bounce, which will
>> > significantly
>> > > > > improves the availability of the MirrorMaker pipeline. In my
>> opinion,
>> > > the
>> > > > > main benefit of the KIP is to avoid unnecessary rebalance if the
>> > > consumer
>> > > > > process can be restarted within soon, which helps performance
>> even if
>> > > > > overhead of state shuffling for a given process is small.
>> > > > >
>> > > > > 2) In order to simplify the KIP reading, can you follow the
>> writeup
>> > > style
>> > > > > of other KIP (e.g. KIP-98) and list the interface change such as
>> new
>> > > > > configs (e.g. registration timeout), new request/response, new
>> > > > AdminClient
>> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
>> some
>> > > of
>> > > > > these are specified in the Proposed Change section which makes it
>> a
>> > bit
>> > > > > inconvenient to understand the new interface that will be exposed
>> to
>> > > > user.
>> > > > > Explanation of the current two-phase rebalance protocol probably
>> can
>> > be
>> > > > > moved out of public interface section.
>> > > > >
>> > > > > 3) There are currently two version of JoinGroupRequest in the KIP
>> and
>> > > > only
>> > > > > one of them has field memberId. This seems confusing.
>> > > > >
>> > > > > 4) It is mentioned in the KIP that "An admin API to force
>> rebalance
>> > > could
>> > > > > be helpful here, but we will make a call once we finished the
>> major
>> > > > > implementation". So this seems to be still an open question in the
>> > > > current
>> > > > > design. We probably want to agree on this before voting for the
>> KIP.
>> > > > >
>> > > > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
>> > you
>> > > > > specify the name of the config key and the default config value?
>> > > Possible
>> > > > > default values include empty string or null (similar to
>> > transaction.id
>> > > > in
>> > > > > producer config).
>> > > > >
>> > > > > 6) Regarding the use of the topic "static_member_map" to persist
>> > member
>> > > > > name map, currently if consumer coordinator broker goes offline,
>> > > > rebalance
>> > > > > is triggered and consumers will try connect to the new
>> coordinator.
>> > If
>> > > > > these consumers can connect to the new coordinator within
>> > > > > max.poll.interval.ms which by default is 5 minutes, given that
>> > broker
>> > > > can
>> > > > > use a deterministic algorithm to determine the partition ->
>> > member_name
>> > > > > mapping, each consumer should get assigned the same set of
>> partitions
>> > > > > without requiring state shuffling. So it is not clear whether we
>> > have a
>> > > > > strong use-case for this new logic. Can you help clarify what is
>> the
>> > > > > benefit of using topic "static_member_map" to persist member name
>> > map?
>> > > > >
>> > > > > 7) Regarding the introduction of the expensionTimeoutMs config,
>> it is
>> > > > > mentioned that "we are using expansion timeout to replace
>> rebalance
>> > > > > timeout, which is configured by max.poll.intervals from client
>> side,
>> > > and
>> > > > > using registration timeout to replace session timeout". Currently
>> the
>> > > > > default max.poll.interval.ms is configured to be 5 minutes and
>> there
>> > > > will
>> > > > > be only one rebalance if all new consumers can join within 5
>> minutes.
>> > > So
>> > > > it
>> > > > > is not clear whether we have a strong use-case for this new
>> config.
>> > Can
>> > > > you
>> > > > > explain what is the benefit of introducing this new config?
>> > > > >
>> > > > > 8) It is mentioned that "To distinguish between previous version
>> of
>> > > > > protocol, we will also increase the join group request version to
>> v4
>> > > when
>> > > > > MEMBER_NAME is set" and "If the broker version is not the latest
>> (<
>> > > v4),
>> > > > > the join group request shall be downgraded to v3 without setting
>> the
>> > > > member
>> > > > > Id". It is probably simpler to just say that this feature is
>> enabled
>> > if
>> > > > > JoinGroupRequest V4 is supported on both client and broker and
>> > > > MEMBER_NAME
>> > > > > is configured with non-empty string.
>> > > > >
>> > > > > 9) It is mentioned that broker may return
>> NO_STATIC_MEMBER_INFO_SET
>> > > error
>> > > > > in OffsetCommitResponse for "commit requests under static
>> > membership".
>> > > > Can
>> > > > > you clarify how broker determines whether the commit request is
>> under
>> > > > > static membership?
>> > > > >
>> > > > > Thanks,
>> > > > > Dong
>> > > > >
>> > > >
>> > > >
>> > > > --
>> > > > -- Guozhang
>> > > >
>> > >
>> >
>> >
>> > --
>> > -Regards,
>> > Mayuresh R. Gharat
>> > (862) 250-7125
>> >
>>
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Do you have a discuss thread for KIP-394 that you mentioned here ?

Thanks,

Mayuresh

On Mon, Nov 26, 2018 at 4:52 AM Boyang Chen <bc...@outlook.com> wrote:

> Hey Dong, thanks for the follow-up here!
>
>
> 1) It is not very clear to the user what is the difference between
> member.name and client.id as both seems to be used to identify the
> consumer. I am wondering if it would be more intuitive to name it
> group.member.name (preferred choice since it matches the current group.id
> config name) or rebalance.member.name to explicitly show that the id is
> solely used for rebalance.
> Great question. I feel `member.name` is enough to explain itself, it
> seems not very
> helpful to make the config name longer. Comparing `name` with `id` gives
> user the
> impression that they have the control over it with customized rule than
> library decided.
>
> 2) In the interface change section it is said that GroupMaxSessionTimeoutMs
> will be changed to 30 minutes. It seems to suggest that we will change the
> default value of this config. It does not seem necessary to increase the
> time of consumer failure detection when user doesn't use static membership.
> Also, say static membership is enabled, then this default config change
> will cause a partition to be unavailable for consumption for 30 minutes if
> there is hard consumer failure, which seems to be worse experience than
> having unnecessary rebalance (when this timeout is small), particularly for
> new users of Kafka. Could you explain more why we should make this change?
> We are not changing the default session timeout value. We are just
> changing the
> cap we are enforcing on the session timeout max value. So this change is
> not affecting
> what kind of membership end user is using, and loosing the cap is giving
> end user
> more flexibility on trade-off between liveness and stability.
>
> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> into one error? It seems that these two errors are currently handled by the
> consumer in the same way. And we don't also don't expect MEMBER_ID_MISMATCH
> to happen. Thus it is not clear what is the benefit of having two errors.
> I agree that we should remove DUPLICATE_STATIC_MEMBER error because with
> the KIP-394<
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member+id+for+initial+join+group+request
> >
> we will automatically fence all join requests with UNKNOWN_MEMBER_ID.
>
> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
> member name which is already in the consumer group, however the member id
> was missing". After a consumer is restarted, it will send a
> JoinGroupRequest with an existing memberName (as the coordinator has not
> expired this member from the memory) and memberId
> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> across consumer restart in the consumer side). Does it mean that
> JoinGroupRequest from a newly restarted consumer will always be rejected
> until the sessionTimeoutMs has passed?
> Same answer as question 3). This part of the logic shall be removed from
> the proposal.
>
> 5) It seems that we always add two methods to the interface
> org.apache.kafka.clients.admin.AdminClient.java, one with options and the
> other without option. Could this be specified in the interface change
> section?
> Sounds good! Added both methods.
>
> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> rebalance? If so, we probably want to specify the command line tool
> interface similar to
>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
> .
> Added the script.
>
> 7) Would it be simpler to replace name "forceStaticRebalance" with
> "invokeConsumerRebalance"? It is not very clear what is the extra meaning
> of world "force" as compared to "trigger" or "invoke". And it seems simpler
> to allows this API to trigger rebalance regardless of whether consumer is
> configured with memberName.
> Sounds good. Right now I feel for both static and dynamic membership it is
> more manageable to introduce the consumer rebalance method through admin
> client API.
>
> 8) It is not very clear how the newly added AdminClient API trigger
> rebalance. For example, does it send request? Can this be explained in the
> KIP?
>
> Sure, I will add more details to the API.
>
>
> Thanks again for the helpful suggestions!
>
>
> Best,
> Boyang
>
> ________________________________
> From: Dong Lin <li...@gmail.com>
> Sent: Saturday, November 24, 2018 2:54 PM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey Boyang,
>
> Thanks for the update! Here are some followup comments:
>
> 1) It is not very clear to the user what is the difference between
> member.name and client.id as both seems to be used to identify the
> consumer. I am wondering if it would be more intuitive to name it
> group.member.name (preferred choice since it matches the current group.id
> config name) or rebalance.member.name to explicitly show that the id is
> solely used for rebalance.
>
> 2) In the interface change section it is said that GroupMaxSessionTimeoutMs
> will be changed to 30 minutes. It seems to suggest that we will change the
> default value of this config. It does not seem necessary to increase the
> time of consumer failure detection when user doesn't use static membership.
> Also, say static membership is enabled, then this default config change
> will cause a partition to be unavailable for consumption for 30 minutes if
> there is hard consumer failure, which seems to be worse experience than
> having unnecessary rebalance (when this timeout is small), particularly for
> new users of Kafka. Could you explain more why we should make this change?
>
> 3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
> into one error? It seems that these two errors are currently handled by the
> consumer in the same way. And we don't also don't expect MEMBER_ID_MISMATCH
> to happen. Thus it is not clear what is the benefit of having two errors.
>
> 4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
> member name which is already in the consumer group, however the member id
> was missing". After a consumer is restarted, it will send a
> JoinGroupRequest with an existing memberName (as the coordinator has not
> expired this member from the memory) and memberId
> = JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
> across consumer restart in the consumer side). Does it mean that
> JoinGroupRequest from a newly restarted consumer will always be rejected
> until the sessionTimeoutMs has passed?
>
> 5) It seems that we always add two methods to the interface
> org.apache.kafka.clients.admin.AdminClient.java, one with options and the
> other without option. Could this be specified in the interface change
> section?
>
> 6) Do we plan to have off-the-shelf command line tool for SRE to trigger
> rebalance? If so, we probably want to specify the command line tool
> interface similar to
>
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
> .
>
> 7) Would it be simpler to replace name "forceStaticRebalance" with
> "invokeConsumerRebalance"? It is not very clear what is the extra meaning
> of world "force" as compared to "trigger" or "invoke". And it seems simpler
> to allows this API to trigger rebalance regardless of whether consumer is
> configured with memberName.
>
> 8) It is not very clear how the newly added AdminClient API trigger
> rebalance. For example, does it send request? Can this be explained in the
> KIP?
>
> Thanks,
> Dong
>
>
> On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > Hey Mayuresh,
> >
> >
> > thanks for your feedbacks! I will try do another checklist here.
> >
> >
> > > By this you mean, even if the application has not called
> > > KafkaConsumer.poll() within session timeout, it will not be sending the
> > > LeaveGroup request, right?
> >
> > Yep it's true, we will prevent client from sending leave group request
> > when they are set with `member.name`.
> >
> >
> > > When is the member.name removed from this map?
> > Good question, we will only kick off member due to session timeout within
> > static membership. Let me update the KIP to clearly assert that.
> >
> > > How is this case (missing member id) handled on the client side? What
> is
> > the application that
> > > is using the KafkaConsumer suppose to do in this scenario?
> > I have extended the two exceptions within join group response V4.
> > Basically I define both corresponding actions to be immediate failing
> > client application, because so far it is unknown what kind of client
> issue
> > could trigger them. After the first version, we will keep enhance the
> error
> > handling logic!
> >
> > > This would mean that it might take more time to detect unowned topic
> > > partitions and may cause delay for applications that perform data
> > mirroring
> > > tasks. I discussed this with our sre and we have a suggestion to make
> > here
> > > as listed below separately.
> > The goal of extending session timeout cap is for users with good client
> > side monitoring tools that could auto-heal the dead consumers very fast.
> So
> > it is optional (and personal) to extend session timeout to a reasonable
> > number with different client scenarios.
> >
> > > you meant remove unjoined members of the group, right ?
> > Yep, there is a typo. Thanks for catching this!
> >
> > > What do you mean by " Internally we would optimize this logic by having
> > > rebalance timeout only in charge of stopping prepare rebalance stage,
> > > without removing non-responsive members immediately." There would not
> be
> > a
> > > full rebalance if the lagging consumer sent a JoinGroup request later,
> > > right ? If yes, can you highlight this in the KIP ?
> > No, there won't be. We want to limit the rebalance timeout functionality
> > to only use as a timer to
> > end prepare rebalance stage. This way, late joining static members will
> > not trigger further rebalance
> > as long as they are within session timeout. I added your highlight to the
> > KIP!
> >
> > > The KIP talks about scale up scenario but its not quite clear how we
> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> > status
> > > "learner" ?. Can you shed more light on how this is handled in the KIP,
> > if
> > > its handled?
> > Updated the KIP: we shall not cover scale up case in 345, because we
> > believe client side could
> > better handle this logic.
> >
> > > I think Jason had brought this up earlier about having a way to say how
> > > many members/consumer hosts are you choosing to be in the consumer
> group.
> > > If we can do this, then in case of mirroring applications we can do
> this
> > :
> > > Lets say we have a mirroring application that consumes from Kafka
> cluster
> > > A and produces to Kafka cluster B.
> > > Depending on the data and the Kafka cluster configuration, Kafka
> service
> > > providers can set a mirroring group saying that it will take, for
> example
> > > 300 consumer hosts/members to achieve the desired throughput and
> latency
> > > for mirroring and can have additional 10 consumer hosts as spare in the
> > > same group.
> > > So when the first 300 members/consumers to join the group will start
> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > The remaining 10 consumer members can sit idle.
> > > The moment one of the consumer (for example: consumer number 54) from
> the
> > > first 300 members go out of the group (crossed session timeout), it
> (the
> > > groupCoordinator) can just assign the topicPartitions from the consumer
> > > member 54 to one of the spare hosts.
> > > Once the consumer member 54 comes back up, it can start as being a part
> > of
> > > the spare pool.
> > > This enables us to have lower session timeouts and low latency
> mirroring,
> > > in cases where the service providers are OK with having spare hosts.
> > > This would mean that we would tolerate n consumer members leaving and
> > > rejoining the group and still provide low latency as long as n <=
> number
> > of
> > > spare consumers.
> > > If there are no spare host available, we can get back to the idea as
> > > described in the KIP.
> > Great idea! In fact on top of static membership we could later introduce
> > APIs to set hard-coded
> > client ids to the group and replace the dead host, or as you proposed to
> > define spare host as
> > what I understood as hot backup. I will put both Jason and your
> > suggestions into a separate section
> > called "Future works". Note that this spare host idea may be also
> solvable
> > through rebalance protocol
> > IMO.
> >
> > Thank you again for the great feedback!
> >
> > Boyang
> > ________________________________
> > From: Boyang Chen <bc...@outlook.com>
> > Sent: Thursday, November 22, 2018 3:39 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hey Dong, sorry for missing your message. I couldn't find your email on
> my
> > thread, so I will just do a checklist here!
> >
> >
> > 1) The motivation currently explicitly states that the goal is to improve
> >
> > performance for heavy state application. It seems that the motivation can
> >
> > be stronger with the following use-case. Currently for MirrorMaker
> cluster
> >
> > with e.g. 100 MirrorMaker processes, it will take a long time to rolling
> >
> > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> >
> > will trigger a rebalance which currently pause the consumption of the all
> >
> > partitions of the MirrorMaker cluster. With the change stated in this
> >
> > patch, as long as a MirrorMaker can restart within the specified timeout
> >
> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> for
> >
> > leader restart) for the entire rolling bounce, which will significantly
> >
> > improves the availability of the MirrorMaker pipeline. In my opinion, the
> >
> > main benefit of the KIP is to avoid unnecessary rebalance if the consumer
> >
> > process can be restarted within soon, which helps performance even if
> >
> > overhead of state shuffling for a given process is small.
> >
> > I just rephrased this part and added it to the KIP. Thanks for making the
> > motivation more solid!
> >
> > 2) In order to simplify the KIP reading, can you follow the writeup style
> > of other KIP (e.g. KIP-98) and list the interface change such as new
> > configs (e.g. registration timeout), new request/response, new
> AdminClient
> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> > these are specified in the Proposed Change section which makes it a bit
> > inconvenient to understand the new interface that will be exposed to
> user.
> > Explanation of the current two-phase rebalance protocol probably can be
> > moved out of public interface section.
> > This is a great suggestion! I just consolidated all the public API
> > changes, and the whole KIP
> > looks much more organized!
> >
> > 3) There are currently two version of JoinGroupRequest in the KIP and
> only
> > one of them has field memberId. This seems confusing.
> > Yep, I already found this issue and fixed it.
> >
> > 4) It is mentioned in the KIP that "An admin API to force rebalance could
> > be helpful here, but we will make a call once we finished the major
> > implementation". So this seems to be still an open question in the
> current
> > design. We probably want to agree on this before voting for the KIP.
> > We have finalized the idea that this API is needed.
> >
> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > specify the name of the config key and the default config value? Possible
> > default values include empty string or null (similar to transaction.id<
> >
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=kkCtxHpsAZpbJZpUc52dtv6ac8UJOx6CQlts3CPjDh8%3D&amp;reserved=0
> >
> > in
> > producer config).
> > I have defined the `member.name` in "New configuration" section.
> >
> > 6) Regarding the use of the topic "static_member_map" to persist member
> > name map, currently if consumer coordinator broker goes offline,
> rebalance
> > is triggered and consumers will try connect to the new coordinator. If
> > these consumers can connect to the new coordinator within
> > max.poll.interval.ms<
> >
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
> >
> > which by default is 5 minutes, given that broker can
> > use a deterministic algorithm to determine the partition -> member_name
> > mapping, each consumer should get assigned the same set of partitions
> > without requiring state shuffling. So it is not clear whether we have a
> > strong use-case for this new logic. Can you help clarify what is the
> > benefit of using topic "static_member_map" to persist member name map?
> > I have discussed with Guozhang offline, and I believe reusing the current
> > `_consumer_offsets`
> > topic is a better and unified solution.
> >
> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > mentioned that "we are using expansion timeout to replace rebalance
> > timeout, which is configured by max.poll.intervals from client side, and
> > using registration timeout to replace session timeout". Currently the
> > default max.poll.interval.ms<
> >
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0
> >
> > is configured to be 5 minutes and there will
> > be only one rebalance if all new consumers can join within 5 minutes. So
> it
> > is not clear whether we have a strong use-case for this new config. Can
> you
> > explain what is the benefit of introducing this new config?
> > Previously our goal is to use expansion timeout as a workaround for
> > triggering multiple
> > rebalances when scaling up members are not joining at the same time. It
> is
> > decided to
> > be addressed by client side protocol change, so we will not introduce
> > expansion timeout.
> >
> > 8) It is mentioned that "To distinguish between previous version of
> > protocol, we will also increase the join group request version to v4 when
> > MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> > the join group request shall be downgraded to v3 without setting the
> member
> > Id". It is probably simpler to just say that this feature is enabled if
> > JoinGroupRequest V4 is supported on both client and broker and
> MEMBER_NAME
> > is configured with non-empty string.
> > Yep, addressed this!
> >
> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> > in OffsetCommitResponse for "commit requests under static membership".
> Can
> > you clarify how broker determines whether the commit request is under
> > static membership?
> >
> > We have agreed that commit request shouldn't be affected by the new
> > membership, thus
> > removing it here. Thanks for catching this!
> >
> > Let me know if you have further suggestions or concerns. Thank you for
> > your valuable feedback
> > to help me design the KIP better! (And I will try to address your
> > feedbacks in next round Mayuresh ??)
> >
> > Best,
> > Boyang
> > ________________________________
> > From: Mayuresh Gharat <gh...@gmail.com>
> > Sent: Wednesday, November 21, 2018 7:50 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for updating the KIP. This is a step good direction for stateful
> > applications and also mirroring applications whose latency is affected
> due
> > to the rebalance issues that we have today.
> >
> > I had a few questions on the current version of the KIP :
> > For the effectiveness of the KIP, consumer with member.name set will
> *not
> > send leave group request* when they go offline
> >
> > > By this you mean, even if the application has not called
> > > KafkaConsumer.poll() within session timeout, it will not be sending the
> > > LeaveGroup request, right?
> > >
> >
> > Broker will maintain an in-memory mapping of {member.name ? member.id}
> to
> > track member uniqueness.
> >
> > > When is the member.name removed from this map?
> > >
> >
> > Member.id must be set if the *member.name <
> >
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=aSTzJlF4GPFGKhi5K7MPZozMn67718GWpqEYxFay%2BZs%3D&amp;reserved=0
> >
> > *is already
> > within the map. Otherwise reply MISSING_MEMBER_ID
> >
> > > How is this case handled on the client side? What is the application
> that
> > > is using the KafkaConsumer suppose to do in this scenario?
> > >
> >
> > Session timeout is the timeout we will trigger rebalance when a member
> goes
> > offline for too long (not sending heartbeat request). To make static
> > membership effective, we should increase the default max session timeout
> to
> > 30 min so that end user could config it freely.
> >
> > > This would mean that it might take more time to detect unowned topic
> > > partitions and may cause delay for applications that perform data
> > mirroring
> > > tasks. I discussed this with our sre and we have a suggestion to make
> > here
> > > as listed below separately.
> > >
> >
> > Currently there is a config called *rebalance timeout* which is
> configured
> > by consumer *max.poll.intervals*. The reason we set it to poll interval
> is
> > because consumer could only send request within the call of poll() and we
> > want to wait sufficient time for the join group request. When reaching
> > rebalance timeout, the group will move towards completingRebalance stage
> > and remove unjoined groups
> >
> > > you meant remove unjoined members of the group, right ?
> > >
> >
> > Currently there is a config called *rebalance timeout* which is
> configured
> > by consumer *max.poll.intervals*. The reason we set it to poll interval
> is
> > because consumer could only send request within the call of poll() and we
> > want to wait sufficient time for the join group request. When reaching
> > rebalance timeout, the group will move towards completingRebalance stage
> > and remove unjoined groups. This is actually conflicting with the design
> of
> > static membership, because those temporarily unavailable members will
> > potentially reattempt the join group and trigger extra rebalances.
> > Internally we would optimize this logic by having rebalance timeout only
> in
> > charge of stopping prepare rebalance stage, without removing
> non-responsive
> > members immediately.
> >
> > > What do you mean by " Internally we would optimize this logic by having
> > > rebalance timeout only in charge of stopping prepare rebalance stage,
> > > without removing non-responsive members immediately." There would not
> be
> > a
> > > full rebalance if the lagging consumer sent a JoinGroup request later,
> > > right ? If yes, can you highlight this in the KIP ?
> > >
> >
> > Scale Up
> >
> > > The KIP talks about scale up scenario but its not quite clear how we
> > > handle it. Are we adding a separate "expansion.timeout" or we adding
> > status
> > > "learner" ?. Can you shed more light on how this is handled in the KIP,
> > if
> > > its handled?
> > >
> >
> >
> > *Discussion*
> > Larger session timeouts causing latency rise for getting data for
> un-owned
> > topic partitions :
> >
> > > I think Jason had brought this up earlier about having a way to say how
> > > many members/consumer hosts are you choosing to be in the consumer
> group.
> > > If we can do this, then in case of mirroring applications we can do
> this
> > :
> > > Lets say we have a mirroring application that consumes from Kafka
> cluster
> > > A and produces to Kafka cluster B.
> > > Depending on the data and the Kafka cluster configuration, Kafka
> service
> > > providers can set a mirroring group saying that it will take, for
> example
> > > 300 consumer hosts/members to achieve the desired throughput and
> latency
> > > for mirroring and can have additional 10 consumer hosts as spare in the
> > > same group.
> > > So when the first 300 members/consumers to join the group will start
> > > mirroring the data from Kafka cluster A to Kafka cluster B.
> > > The remaining 10 consumer members can sit idle.
> > > The moment one of the consumer (for example: consumer number 54) from
> the
> > > first 300 members go out of the group (crossed session timeout), it
> (the
> > > groupCoordinator) can just assign the topicPartitions from the consumer
> > > member 54 to one of the spare hosts.
> > > Once the consumer member 54 comes back up, it can start as being a part
> > of
> > > the spare pool.
> > > This enables us to have lower session timeouts and low latency
> mirroring,
> > > in cases where the service providers are OK with having spare hosts.
> > > This would mean that we would tolerate n consumer members leaving and
> > > rejoining the group and still provide low latency as long as n <=
> number
> > of
> > > spare consumers.
> > > If there are no spare host available, we can get back to the idea as
> > > described in the KIP.
> > >
> >
> > Thanks,
> >
> > Mayuresh
> >
> >
> >
> >
> >
> > On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> > konstantine@confluent.io> wrote:
> >
> > > Hi Boyang.
> > >
> > > Thanks for preparing this KIP! It is making good progress and will be a
> > > great improvement for stateful Kafka applications.
> > >
> > > Apologies for my late reply, I was away for a while. Lots of great
> > comments
> > > so far, so I'll probably second most of them in what I suggest below at
> > > this point.
> > >
> > > When I first read the KIP, I wanted to start at the end with something
> > that
> > > wasn't highlighted a lot. That was the topic related to handling
> > duplicate
> > > members. I see now that the initial suggestion of handling this
> situation
> > > during offset commit has been removed, and I agree with that. Issues
> > > related to membership seem to be handled better when the member joins
> the
> > > group rather than when it tries to commit offsets. This also simplifies
> > how
> > > many request types need to change in order to incorporate the new
> member
> > > name field.
> > >
> > > I also agree with what Jason and Guozhang have said regarding timeouts.
> > > Although semantically, it's easier to think of every operation having
> its
> > > own timeout, operationally this can become a burden. Thus,
> consolidation
> > > seems preferable here. The definition of embedded protocols on top of
> the
> > > base group membership protocol for rebalancing gives enough flexibility
> > to
> > > address such needs in each client component separately.
> > >
> > > Finally, some minor comments:
> > > In a few places the new/proposed changes are referred to as "current".
> > > Which is a bit confusing considering that there is a protocol in place
> > > already, and by "current" someone might understand the existing one.
> I'd
> > > recommend using new/proposed or equivalent when referring to changes
> > > introduced with KIP-345 and current/existing or equivalent when
> referring
> > > to existing behavior.
> > >
> > > There's the following sentence in the "Public Interfaces" section:
> > > "Since for many stateful consumer/stream applications, the state
> > shuffling
> > > is more painful than short time partial unavailability."
> > > However, my understanding is that the changes proposed with KIP-345
> will
> > > not exploit any partial availability. A suggestion for dealing with
> > > temporary imbalances has been made in "Incremental Cooperative
> > Rebalancing"
> > > which can work well with KIP-345, but here I don't see proposed changes
> > > that suggest that some resources (e.g. partitions) will keep being used
> > > while others will not be utilized. Thus, you might want to adjust this
> > > sentence. Correct me if I'm missing something related to that.
> > >
> > > In the rejected alternatives, under point 2) I read "we can copy the
> > member
> > > id to the config files". I believe it means to say "member name" unless
> > I'm
> > > missing something about reusing member ids. Also below I read: "By
> > allowing
> > > consumers to optionally specifying a member id" which probably implies
> > > "member name" again. In a sense this section highlights a potential
> > > confusion between member name and member id. I wonder if we could come
> up
> > > with a better term for the new field. StaticTag, StaticLabel, or even
> > > StaticName are some suggestions that could potentially help with
> > confusion
> > > between MemberId and MemberName and what corresponds to what. But I
> > > wouldn't like to disrupt the discussion with naming conventions too
> much
> > at
> > > this point. I just mention it here as a thought.
> > >
> > > Looking forward to see the final details of this KIP. Great work so
> far!
> > >
> > > Konstantine
> > >
> > >
> > > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > > > Thanks Guozhang for the great summary here, and I have been following
> > up
> > > > the action items here.
> > > >
> > > >
> > > >   1.  I already updated the KIP to remove the expansion timeout and
> > > > registration timeout. Great to see them being addressed in client
> side!
> > > >   2.  I double checked the design and I believe that it is ok to have
> > > both
> > > > static member and dynamic member co-exist in the same group. So the
> > > upgrade
> > > > shouldn't be destructive and we are removing the two membership
> > protocol
> > > > switching APIs.
> > > >   3.  I only have question about this one. I'm still reading the
> > > KafkaApis
> > > > code here. Should I just use the same authorization logic for
> > > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > > >   4.  I'm very excited to see this work with K8! Like you suggested,
> > this
> > > > feature could be better addressed in a separate KIP because it is
> > pretty
> > > > independent. I could start drafting the KIP once the current proposal
> > is
> > > > approved.
> > > >   5.  I believe that we don't need fencing in offset commit request,
> > > since
> > > > duplicate member.name issue could be handled by join group request.
> We
> > > > shall reject join group with known member name but no member id
> (which
> > > > means we already have an active member using this identity).
> > > >   6.  I agree to remove that internal config once we move forward
> with
> > > > static membership. And I already removed the entire section from the
> > KIP.
> > > >
> > > > Let me know if you have other concerns.
> > > >
> > > > Best,
> > > > Boyang
> > > > ________________________________
> > > > From: Guozhang Wang <wa...@gmail.com>
> > > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > > To: dev
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Hello Boyang,
> > > >
> > > > Thanks a lot for the KIP! It is a great write-up and I appreciate
> your
> > > > patience answering to the feedbacks from the community. I'd like to
> add
> > > my
> > > > 2cents here:
> > > >
> > > > 1. By introducing another two timeout configs, registration_timeout
> and
> > > > expansion_timeout, we are effectively having four timeout configs:
> > > session
> > > > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> > > client
> > > > side), and these two. Interplaying these timeout configs can be quite
> > > hard
> > > > for users with such complexity, and hence I'm wondering if we can
> > > simplify
> > > > the situation with as less possible timeout configs as possible. Here
> > is
> > > a
> > > > concrete suggestion I'd like propose:
> > > >
> > > > 1.a) Instead of introducing a registration_timeout in addition to the
> > > > session_timeout for static members, we can just reuse the
> > session_timeout
> > > > and ask users to set it to a larger value when they are upgrading a
> > > dynamic
> > > > client to a static client by setting the "member.name" at the same
> > time.
> > > > By
> > > > default, the broker-side min.session.timeout is 6 seconds and
> > > > max.session.timeout is 5 minutes, which seems reasonable to me (we
> can
> > of
> > > > course modify this broker config to enlarge the valid interval if we
> > want
> > > > in practice). And then we should also consider removing the condition
> > for
> > > > marking a client as failed if the rebalance timeout has reached while
> > the
> > > > JoinGroup was not received, so that the semantics of session_timeout
> > and
> > > > rebalance_timeout are totally separated: the former is only used to
> > > > determine if a consumer member of the group should be marked as
> failed
> > > and
> > > > kicked out of the group, and the latter is only used to determine the
> > > > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> > > other
> > > > words if a member did not send the JoinGroup in time of the
> > > > rebalance_timeout, we still include it in the new generation of the
> > group
> > > > and use its old subscription info to send to leader for assignment.
> > Later
> > > > if the member came back with HeartBeat request, we can still follow
> the
> > > > normal path to bring it to the latest generation while checking that
> > its
> > > > sent JoinGroup request contains the same subscription info as we used
> > to
> > > > assign the partitions previously (which should be likely the case in
> > > > practice). In addition, we should let static members to not send the
> > > > LeaveGroup request when it is gracefully shutdown, so that a static
> > > member
> > > > can only be leaving the group if its session has timed out, OR it has
> > > been
> > > > indicated to not exist in the group any more (details below).
> > > >
> > > > 1.b) We have a parallel discussion about Incremental Cooperative
> > > > Rebalancing, in which we will encode the "when to rebalance" logic at
> > the
> > > > application level, instead of at the protocol level. By doing this we
> > can
> > > > also enable a few other optimizations, e.g. at the Streams level to
> > first
> > > > build up the state store as standby tasks and then trigger a second
> > > > rebalance to actually migrate the active tasks while keeping the
> actual
> > > > rebalance latency and hence unavailability window to be small (
> > > >
> > > >
> > >
> >
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=ZJlsB%2FHLhZykd9mtu5CINNNEqMBvX75bdhqR3IlxGI8%3D&amp;reserved=0
> > > ).
> > > > I'd propose we align
> > > > KIP-345 along with this idea, and hence do not add the
> > expansion_timeout
> > > as
> > > > part of the protocol layer, but only do that at the application's
> > > > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > > > deprecate the "*group.initial.rebalance.delay.ms
> > > > <
> > > >
> > >
> >
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=DDhjV41nPU3euYCQ3w8WPENuw9fPB6ah2j6rF0JjRBg%3D&amp;reserved=0
> > > >*"
> > > > though as part of this KIP
> > > > since we have discussed about its limit and think it is actually not
> a
> > > very
> > > > good design and could be replaced with client-side logic above.
> > > >
> > > >
> > > > 2. I'd like to see your thoughts on the upgrade path for this KIP.
> More
> > > > specifically, let's say after we have upgraded broker version to be
> > able
> > > to
> > > > recognize the new versions of JoinGroup request and the admin
> requests,
> > > how
> > > > should we upgrade the clients and enable static groups? On top of my
> > head
> > > > if we do a rolling bounce in which we set the member.name config as
> > well
> > > > as
> > > > optionally increase the session.timeout config when we bounce each
> > > > instance, then during this rolling bounces we will have a group
> > contained
> > > > with both dynamic members and static members. It means that we should
> > > have
> > > > the group to allow such scenario (i.e. we cannot reject JoinGroup
> > > requests
> > > > from dynamic members), and hence the "member.name" -> "member.id"
> > > mapping
> > > > will only be partial at this scenario. Also could you describe if the
> > > > upgrade to the first version that support this feature would ever get
> > any
> > > > benefits, or only the future upgrade path for rolling bounces could
> get
> > > > benefits out of this feature?
> > > >
> > > > If that's the case and we will do 1) as suggested above, do we still
> > need
> > > > the enableStaticMembership and enableDynamicMembership admin requests
> > any
> > > > more? Seems it is not necessary any more as we will only have the
> > notion
> > > of
> > > > "dynamic or static members" that can co-exist in a group while there
> no
> > > > notion of "dynamic or static groups", and hence these two requests
> are
> > > not
> > > > needed anymore.
> > > >
> > > >
> > > > 3. We need to briefly talk about the implications for ACL as we
> > introduce
> > > > new admin requests that are related to a specific group.id. For
> > example,
> > > > we
> > > > need to make sure that whoever created the group or joined the group
> > can
> > > > actually send admin requests for the group, otherwise the application
> > > > owners need to bother the Kafka operators on a multi-tenant cluster
> > every
> > > > time they want to send any admin requests for their groups which
> would
> > be
> > > > an operational nightmare.
> > > >
> > > >
> > > > 4. I like Jason's suggestion of adding an optional field for the list
> > of
> > > > member names, and I'm wondering if that can be done as part of the
> > > > forceStaticRebalance request: i.e. by passing a list of members, we
> > will
> > > > enforce a rebalance immediately since it indicates that some static
> > > member
> > > > will be officially kicked out of the group and some new static
> members
> > > may
> > > > be added. So back to 1.a) above, a static member can only be kicked
> out
> > > of
> > > > the group if a) its session (arguably long period of time) has timed
> > out,
> > > > and b) this admin request explicitly state that it is no longer part
> of
> > > the
> > > > group. As for execution I'm fine with keeping it as a future work of
> > this
> > > > KIP if you'd like to make its scope smaller.
> > > >
> > > > Following are minor comments:
> > > >
> > > > 5. I'm not sure if we need to include "member.name" as part of the
> > > > OffsetCommitRequest for fencing purposes, as I think the memberId
> plus
> > > the
> > > > generation number should be sufficient for fencing even with static
> > > > members.
> > > >
> > > > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > > >
> > > >
> > > > Guozhang
> > > >
> > > >
> > > >
> > > >
> > > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com>
> wrote:
> > > >
> > > > > Hey Boyang,
> > > > >
> > > > > Thanks for the proposal! This is very useful. I have some comments
> > > below:
> > > > >
> > > > > 1) The motivation currently explicitly states that the goal is to
> > > improve
> > > > > performance for heavy state application. It seems that the
> motivation
> > > can
> > > > > be stronger with the following use-case. Currently for MirrorMaker
> > > > cluster
> > > > > with e.g. 100 MirrorMaker processes, it will take a long time to
> > > rolling
> > > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> > restart
> > > > > will trigger a rebalance which currently pause the consumption of
> the
> > > all
> > > > > partitions of the MirrorMaker cluster. With the change stated in
> this
> > > > > patch, as long as a MirrorMaker can restart within the specified
> > > timeout
> > > > > (e.g. 2 minutes), then we only need constant number of rebalance
> > (e.g.
> > > > for
> > > > > leader restart) for the entire rolling bounce, which will
> > significantly
> > > > > improves the availability of the MirrorMaker pipeline. In my
> opinion,
> > > the
> > > > > main benefit of the KIP is to avoid unnecessary rebalance if the
> > > consumer
> > > > > process can be restarted within soon, which helps performance even
> if
> > > > > overhead of state shuffling for a given process is small.
> > > > >
> > > > > 2) In order to simplify the KIP reading, can you follow the writeup
> > > style
> > > > > of other KIP (e.g. KIP-98) and list the interface change such as
> new
> > > > > configs (e.g. registration timeout), new request/response, new
> > > > AdminClient
> > > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently
> some
> > > of
> > > > > these are specified in the Proposed Change section which makes it a
> > bit
> > > > > inconvenient to understand the new interface that will be exposed
> to
> > > > user.
> > > > > Explanation of the current two-phase rebalance protocol probably
> can
> > be
> > > > > moved out of public interface section.
> > > > >
> > > > > 3) There are currently two version of JoinGroupRequest in the KIP
> and
> > > > only
> > > > > one of them has field memberId. This seems confusing.
> > > > >
> > > > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> > > could
> > > > > be helpful here, but we will make a call once we finished the major
> > > > > implementation". So this seems to be still an open question in the
> > > > current
> > > > > design. We probably want to agree on this before voting for the
> KIP.
> > > > >
> > > > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> > you
> > > > > specify the name of the config key and the default config value?
> > > Possible
> > > > > default values include empty string or null (similar to
> > transaction.id
> > > > in
> > > > > producer config).
> > > > >
> > > > > 6) Regarding the use of the topic "static_member_map" to persist
> > member
> > > > > name map, currently if consumer coordinator broker goes offline,
> > > > rebalance
> > > > > is triggered and consumers will try connect to the new coordinator.
> > If
> > > > > these consumers can connect to the new coordinator within
> > > > > max.poll.interval.ms which by default is 5 minutes, given that
> > broker
> > > > can
> > > > > use a deterministic algorithm to determine the partition ->
> > member_name
> > > > > mapping, each consumer should get assigned the same set of
> partitions
> > > > > without requiring state shuffling. So it is not clear whether we
> > have a
> > > > > strong use-case for this new logic. Can you help clarify what is
> the
> > > > > benefit of using topic "static_member_map" to persist member name
> > map?
> > > > >
> > > > > 7) Regarding the introduction of the expensionTimeoutMs config, it
> is
> > > > > mentioned that "we are using expansion timeout to replace rebalance
> > > > > timeout, which is configured by max.poll.intervals from client
> side,
> > > and
> > > > > using registration timeout to replace session timeout". Currently
> the
> > > > > default max.poll.interval.ms is configured to be 5 minutes and
> there
> > > > will
> > > > > be only one rebalance if all new consumers can join within 5
> minutes.
> > > So
> > > > it
> > > > > is not clear whether we have a strong use-case for this new config.
> > Can
> > > > you
> > > > > explain what is the benefit of introducing this new config?
> > > > >
> > > > > 8) It is mentioned that "To distinguish between previous version of
> > > > > protocol, we will also increase the join group request version to
> v4
> > > when
> > > > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> > > v4),
> > > > > the join group request shall be downgraded to v3 without setting
> the
> > > > member
> > > > > Id". It is probably simpler to just say that this feature is
> enabled
> > if
> > > > > JoinGroupRequest V4 is supported on both client and broker and
> > > > MEMBER_NAME
> > > > > is configured with non-empty string.
> > > > >
> > > > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> > > error
> > > > > in OffsetCommitResponse for "commit requests under static
> > membership".
> > > > Can
> > > > > you clarify how broker determines whether the commit request is
> under
> > > > > static membership?
> > > > >
> > > > > Thanks,
> > > > > Dong
> > > > >
> > > >
> > > >
> > > > --
> > > > -- Guozhang
> > > >
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Hey Dong, thanks for the follow-up here!


1) It is not very clear to the user what is the difference between
member.name and client.id as both seems to be used to identify the
consumer. I am wondering if it would be more intuitive to name it
group.member.name (preferred choice since it matches the current group.id
config name) or rebalance.member.name to explicitly show that the id is
solely used for rebalance.
Great question. I feel `member.name` is enough to explain itself, it seems not very
helpful to make the config name longer. Comparing `name` with `id` gives user the
impression that they have the control over it with customized rule than library decided.

2) In the interface change section it is said that GroupMaxSessionTimeoutMs
will be changed to 30 minutes. It seems to suggest that we will change the
default value of this config. It does not seem necessary to increase the
time of consumer failure detection when user doesn't use static membership.
Also, say static membership is enabled, then this default config change
will cause a partition to be unavailable for consumption for 30 minutes if
there is hard consumer failure, which seems to be worse experience than
having unnecessary rebalance (when this timeout is small), particularly for
new users of Kafka. Could you explain more why we should make this change?
We are not changing the default session timeout value. We are just changing the
cap we are enforcing on the session timeout max value. So this change is not affecting
what kind of membership end user is using, and loosing the cap is giving end user
more flexibility on trade-off between liveness and stability.

3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
into one error? It seems that these two errors are currently handled by the
consumer in the same way. And we don't also don't expect MEMBER_ID_MISMATCH
to happen. Thus it is not clear what is the benefit of having two errors.
I agree that we should remove DUPLICATE_STATIC_MEMBER error because with the KIP-394<https://cwiki.apache.org/confluence/display/KAFKA/KIP-394%3A+Require+member+id+for+initial+join+group+request>
we will automatically fence all join requests with UNKNOWN_MEMBER_ID.

4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
member name which is already in the consumer group, however the member id
was missing". After a consumer is restarted, it will send a
JoinGroupRequest with an existing memberName (as the coordinator has not
expired this member from the memory) and memberId
= JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
across consumer restart in the consumer side). Does it mean that
JoinGroupRequest from a newly restarted consumer will always be rejected
until the sessionTimeoutMs has passed?
Same answer as question 3). This part of the logic shall be removed from the proposal.

5) It seems that we always add two methods to the interface
org.apache.kafka.clients.admin.AdminClient.java, one with options and the
other without option. Could this be specified in the interface change
section?
Sounds good! Added both methods.

6) Do we plan to have off-the-shelf command line tool for SRE to trigger
rebalance? If so, we probably want to specify the command line tool
interface similar to
https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
.
Added the script.

7) Would it be simpler to replace name "forceStaticRebalance" with
"invokeConsumerRebalance"? It is not very clear what is the extra meaning
of world "force" as compared to "trigger" or "invoke". And it seems simpler
to allows this API to trigger rebalance regardless of whether consumer is
configured with memberName.
Sounds good. Right now I feel for both static and dynamic membership it is
more manageable to introduce the consumer rebalance method through admin
client API.

8) It is not very clear how the newly added AdminClient API trigger
rebalance. For example, does it send request? Can this be explained in the
KIP?

Sure, I will add more details to the API.


Thanks again for the helpful suggestions!


Best,
Boyang

________________________________
From: Dong Lin <li...@gmail.com>
Sent: Saturday, November 24, 2018 2:54 PM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hey Boyang,

Thanks for the update! Here are some followup comments:

1) It is not very clear to the user what is the difference between
member.name and client.id as both seems to be used to identify the
consumer. I am wondering if it would be more intuitive to name it
group.member.name (preferred choice since it matches the current group.id
config name) or rebalance.member.name to explicitly show that the id is
solely used for rebalance.

2) In the interface change section it is said that GroupMaxSessionTimeoutMs
will be changed to 30 minutes. It seems to suggest that we will change the
default value of this config. It does not seem necessary to increase the
time of consumer failure detection when user doesn't use static membership.
Also, say static membership is enabled, then this default config change
will cause a partition to be unavailable for consumption for 30 minutes if
there is hard consumer failure, which seems to be worse experience than
having unnecessary rebalance (when this timeout is small), particularly for
new users of Kafka. Could you explain more why we should make this change?

3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
into one error? It seems that these two errors are currently handled by the
consumer in the same way. And we don't also don't expect MEMBER_ID_MISMATCH
to happen. Thus it is not clear what is the benefit of having two errors.

4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
member name which is already in the consumer group, however the member id
was missing". After a consumer is restarted, it will send a
JoinGroupRequest with an existing memberName (as the coordinator has not
expired this member from the memory) and memberId
= JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
across consumer restart in the consumer side). Does it mean that
JoinGroupRequest from a newly restarted consumer will always be rejected
until the sessionTimeoutMs has passed?

5) It seems that we always add two methods to the interface
org.apache.kafka.clients.admin.AdminClient.java, one with options and the
other without option. Could this be specified in the interface change
section?

6) Do we plan to have off-the-shelf command line tool for SRE to trigger
rebalance? If so, we probably want to specify the command line tool
interface similar to
https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-113%253A%2BSupport%2Breplicas%2Bmovement%2Bbetween%2Blog%2Bdirectories%23KIP-113%3ASupportreplicasmovementbetweenlogdirectories-Scripts&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=%2BNasMFlJf9rEJc9iDfndcyxA4%2BGWieS1azSKbtdGRW4%3D&amp;reserved=0
.

7) Would it be simpler to replace name "forceStaticRebalance" with
"invokeConsumerRebalance"? It is not very clear what is the extra meaning
of world "force" as compared to "trigger" or "invoke". And it seems simpler
to allows this API to trigger rebalance regardless of whether consumer is
configured with memberName.

8) It is not very clear how the newly added AdminClient API trigger
rebalance. For example, does it send request? Can this be explained in the
KIP?

Thanks,
Dong


On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com> wrote:

> Hey Mayuresh,
>
>
> thanks for your feedbacks! I will try do another checklist here.
>
>
> > By this you mean, even if the application has not called
> > KafkaConsumer.poll() within session timeout, it will not be sending the
> > LeaveGroup request, right?
>
> Yep it's true, we will prevent client from sending leave group request
> when they are set with `member.name`.
>
>
> > When is the member.name removed from this map?
> Good question, we will only kick off member due to session timeout within
> static membership. Let me update the KIP to clearly assert that.
>
> > How is this case (missing member id) handled on the client side? What is
> the application that
> > is using the KafkaConsumer suppose to do in this scenario?
> I have extended the two exceptions within join group response V4.
> Basically I define both corresponding actions to be immediate failing
> client application, because so far it is unknown what kind of client issue
> could trigger them. After the first version, we will keep enhance the error
> handling logic!
>
> > This would mean that it might take more time to detect unowned topic
> > partitions and may cause delay for applications that perform data
> mirroring
> > tasks. I discussed this with our sre and we have a suggestion to make
> here
> > as listed below separately.
> The goal of extending session timeout cap is for users with good client
> side monitoring tools that could auto-heal the dead consumers very fast. So
> it is optional (and personal) to extend session timeout to a reasonable
> number with different client scenarios.
>
> > you meant remove unjoined members of the group, right ?
> Yep, there is a typo. Thanks for catching this!
>
> > What do you mean by " Internally we would optimize this logic by having
> > rebalance timeout only in charge of stopping prepare rebalance stage,
> > without removing non-responsive members immediately." There would not be
> a
> > full rebalance if the lagging consumer sent a JoinGroup request later,
> > right ? If yes, can you highlight this in the KIP ?
> No, there won't be. We want to limit the rebalance timeout functionality
> to only use as a timer to
> end prepare rebalance stage. This way, late joining static members will
> not trigger further rebalance
> as long as they are within session timeout. I added your highlight to the
> KIP!
>
> > The KIP talks about scale up scenario but its not quite clear how we
> > handle it. Are we adding a separate "expansion.timeout" or we adding
> status
> > "learner" ?. Can you shed more light on how this is handled in the KIP,
> if
> > its handled?
> Updated the KIP: we shall not cover scale up case in 345, because we
> believe client side could
> better handle this logic.
>
> > I think Jason had brought this up earlier about having a way to say how
> > many members/consumer hosts are you choosing to be in the consumer group.
> > If we can do this, then in case of mirroring applications we can do this
> :
> > Lets say we have a mirroring application that consumes from Kafka cluster
> > A and produces to Kafka cluster B.
> > Depending on the data and the Kafka cluster configuration, Kafka service
> > providers can set a mirroring group saying that it will take, for example
> > 300 consumer hosts/members to achieve the desired throughput and latency
> > for mirroring and can have additional 10 consumer hosts as spare in the
> > same group.
> > So when the first 300 members/consumers to join the group will start
> > mirroring the data from Kafka cluster A to Kafka cluster B.
> > The remaining 10 consumer members can sit idle.
> > The moment one of the consumer (for example: consumer number 54) from the
> > first 300 members go out of the group (crossed session timeout), it (the
> > groupCoordinator) can just assign the topicPartitions from the consumer
> > member 54 to one of the spare hosts.
> > Once the consumer member 54 comes back up, it can start as being a part
> of
> > the spare pool.
> > This enables us to have lower session timeouts and low latency mirroring,
> > in cases where the service providers are OK with having spare hosts.
> > This would mean that we would tolerate n consumer members leaving and
> > rejoining the group and still provide low latency as long as n <= number
> of
> > spare consumers.
> > If there are no spare host available, we can get back to the idea as
> > described in the KIP.
> Great idea! In fact on top of static membership we could later introduce
> APIs to set hard-coded
> client ids to the group and replace the dead host, or as you proposed to
> define spare host as
> what I understood as hot backup. I will put both Jason and your
> suggestions into a separate section
> called "Future works". Note that this spare host idea may be also solvable
> through rebalance protocol
> IMO.
>
> Thank you again for the great feedback!
>
> Boyang
> ________________________________
> From: Boyang Chen <bc...@outlook.com>
> Sent: Thursday, November 22, 2018 3:39 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey Dong, sorry for missing your message. I couldn't find your email on my
> thread, so I will just do a checklist here!
>
>
> 1) The motivation currently explicitly states that the goal is to improve
>
> performance for heavy state application. It seems that the motivation can
>
> be stronger with the following use-case. Currently for MirrorMaker cluster
>
> with e.g. 100 MirrorMaker processes, it will take a long time to rolling
>
> bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
>
> will trigger a rebalance which currently pause the consumption of the all
>
> partitions of the MirrorMaker cluster. With the change stated in this
>
> patch, as long as a MirrorMaker can restart within the specified timeout
>
> (e.g. 2 minutes), then we only need constant number of rebalance (e.g. for
>
> leader restart) for the entire rolling bounce, which will significantly
>
> improves the availability of the MirrorMaker pipeline. In my opinion, the
>
> main benefit of the KIP is to avoid unnecessary rebalance if the consumer
>
> process can be restarted within soon, which helps performance even if
>
> overhead of state shuffling for a given process is small.
>
> I just rephrased this part and added it to the KIP. Thanks for making the
> motivation more solid!
>
> 2) In order to simplify the KIP reading, can you follow the writeup style
> of other KIP (e.g. KIP-98) and list the interface change such as new
> configs (e.g. registration timeout), new request/response, new AdminClient
> API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> these are specified in the Proposed Change section which makes it a bit
> inconvenient to understand the new interface that will be exposed to user.
> Explanation of the current two-phase rebalance protocol probably can be
> moved out of public interface section.
> This is a great suggestion! I just consolidated all the public API
> changes, and the whole KIP
> looks much more organized!
>
> 3) There are currently two version of JoinGroupRequest in the KIP and only
> one of them has field memberId. This seems confusing.
> Yep, I already found this issue and fixed it.
>
> 4) It is mentioned in the KIP that "An admin API to force rebalance could
> be helpful here, but we will make a call once we finished the major
> implementation". So this seems to be still an open question in the current
> design. We probably want to agree on this before voting for the KIP.
> We have finalized the idea that this API is needed.
>
> 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> specify the name of the config key and the default config value? Possible
> default values include empty string or null (similar to transaction.id<
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=kkCtxHpsAZpbJZpUc52dtv6ac8UJOx6CQlts3CPjDh8%3D&amp;reserved=0>
> in
> producer config).
> I have defined the `member.name` in "New configuration" section.
>
> 6) Regarding the use of the topic "static_member_map" to persist member
> name map, currently if consumer coordinator broker goes offline, rebalance
> is triggered and consumers will try connect to the new coordinator. If
> these consumers can connect to the new coordinator within
> max.poll.interval.ms<
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0>
> which by default is 5 minutes, given that broker can
> use a deterministic algorithm to determine the partition -> member_name
> mapping, each consumer should get assigned the same set of partitions
> without requiring state shuffling. So it is not clear whether we have a
> strong use-case for this new logic. Can you help clarify what is the
> benefit of using topic "static_member_map" to persist member name map?
> I have discussed with Guozhang offline, and I believe reusing the current
> `_consumer_offsets`
> topic is a better and unified solution.
>
> 7) Regarding the introduction of the expensionTimeoutMs config, it is
> mentioned that "we are using expansion timeout to replace rebalance
> timeout, which is configured by max.poll.intervals from client side, and
> using registration timeout to replace session timeout". Currently the
> default max.poll.interval.ms<
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=By6s977ocGSm%2FF5dSvUVtyPM%2B2OUt0XzFMWRHWaoVVk%3D&amp;reserved=0>
> is configured to be 5 minutes and there will
> be only one rebalance if all new consumers can join within 5 minutes. So it
> is not clear whether we have a strong use-case for this new config. Can you
> explain what is the benefit of introducing this new config?
> Previously our goal is to use expansion timeout as a workaround for
> triggering multiple
> rebalances when scaling up members are not joining at the same time. It is
> decided to
> be addressed by client side protocol change, so we will not introduce
> expansion timeout.
>
> 8) It is mentioned that "To distinguish between previous version of
> protocol, we will also increase the join group request version to v4 when
> MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> the join group request shall be downgraded to v3 without setting the member
> Id". It is probably simpler to just say that this feature is enabled if
> JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
> is configured with non-empty string.
> Yep, addressed this!
>
> 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> in OffsetCommitResponse for "commit requests under static membership". Can
> you clarify how broker determines whether the commit request is under
> static membership?
>
> We have agreed that commit request shouldn't be affected by the new
> membership, thus
> removing it here. Thanks for catching this!
>
> Let me know if you have further suggestions or concerns. Thank you for
> your valuable feedback
> to help me design the KIP better! (And I will try to address your
> feedbacks in next round Mayuresh ??)
>
> Best,
> Boyang
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Wednesday, November 21, 2018 7:50 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for updating the KIP. This is a step good direction for stateful
> applications and also mirroring applications whose latency is affected due
> to the rebalance issues that we have today.
>
> I had a few questions on the current version of the KIP :
> For the effectiveness of the KIP, consumer with member.name set will *not
> send leave group request* when they go offline
>
> > By this you mean, even if the application has not called
> > KafkaConsumer.poll() within session timeout, it will not be sending the
> > LeaveGroup request, right?
> >
>
> Broker will maintain an in-memory mapping of {member.name ? member.id} to
> track member uniqueness.
>
> > When is the member.name removed from this map?
> >
>
> Member.id must be set if the *member.name <
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=aSTzJlF4GPFGKhi5K7MPZozMn67718GWpqEYxFay%2BZs%3D&amp;reserved=0>
> *is already
> within the map. Otherwise reply MISSING_MEMBER_ID
>
> > How is this case handled on the client side? What is the application that
> > is using the KafkaConsumer suppose to do in this scenario?
> >
>
> Session timeout is the timeout we will trigger rebalance when a member goes
> offline for too long (not sending heartbeat request). To make static
> membership effective, we should increase the default max session timeout to
> 30 min so that end user could config it freely.
>
> > This would mean that it might take more time to detect unowned topic
> > partitions and may cause delay for applications that perform data
> mirroring
> > tasks. I discussed this with our sre and we have a suggestion to make
> here
> > as listed below separately.
> >
>
> Currently there is a config called *rebalance timeout* which is configured
> by consumer *max.poll.intervals*. The reason we set it to poll interval is
> because consumer could only send request within the call of poll() and we
> want to wait sufficient time for the join group request. When reaching
> rebalance timeout, the group will move towards completingRebalance stage
> and remove unjoined groups
>
> > you meant remove unjoined members of the group, right ?
> >
>
> Currently there is a config called *rebalance timeout* which is configured
> by consumer *max.poll.intervals*. The reason we set it to poll interval is
> because consumer could only send request within the call of poll() and we
> want to wait sufficient time for the join group request. When reaching
> rebalance timeout, the group will move towards completingRebalance stage
> and remove unjoined groups. This is actually conflicting with the design of
> static membership, because those temporarily unavailable members will
> potentially reattempt the join group and trigger extra rebalances.
> Internally we would optimize this logic by having rebalance timeout only in
> charge of stopping prepare rebalance stage, without removing non-responsive
> members immediately.
>
> > What do you mean by " Internally we would optimize this logic by having
> > rebalance timeout only in charge of stopping prepare rebalance stage,
> > without removing non-responsive members immediately." There would not be
> a
> > full rebalance if the lagging consumer sent a JoinGroup request later,
> > right ? If yes, can you highlight this in the KIP ?
> >
>
> Scale Up
>
> > The KIP talks about scale up scenario but its not quite clear how we
> > handle it. Are we adding a separate "expansion.timeout" or we adding
> status
> > "learner" ?. Can you shed more light on how this is handled in the KIP,
> if
> > its handled?
> >
>
>
> *Discussion*
> Larger session timeouts causing latency rise for getting data for un-owned
> topic partitions :
>
> > I think Jason had brought this up earlier about having a way to say how
> > many members/consumer hosts are you choosing to be in the consumer group.
> > If we can do this, then in case of mirroring applications we can do this
> :
> > Lets say we have a mirroring application that consumes from Kafka cluster
> > A and produces to Kafka cluster B.
> > Depending on the data and the Kafka cluster configuration, Kafka service
> > providers can set a mirroring group saying that it will take, for example
> > 300 consumer hosts/members to achieve the desired throughput and latency
> > for mirroring and can have additional 10 consumer hosts as spare in the
> > same group.
> > So when the first 300 members/consumers to join the group will start
> > mirroring the data from Kafka cluster A to Kafka cluster B.
> > The remaining 10 consumer members can sit idle.
> > The moment one of the consumer (for example: consumer number 54) from the
> > first 300 members go out of the group (crossed session timeout), it (the
> > groupCoordinator) can just assign the topicPartitions from the consumer
> > member 54 to one of the spare hosts.
> > Once the consumer member 54 comes back up, it can start as being a part
> of
> > the spare pool.
> > This enables us to have lower session timeouts and low latency mirroring,
> > in cases where the service providers are OK with having spare hosts.
> > This would mean that we would tolerate n consumer members leaving and
> > rejoining the group and still provide low latency as long as n <= number
> of
> > spare consumers.
> > If there are no spare host available, we can get back to the idea as
> > described in the KIP.
> >
>
> Thanks,
>
> Mayuresh
>
>
>
>
>
> On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> konstantine@confluent.io> wrote:
>
> > Hi Boyang.
> >
> > Thanks for preparing this KIP! It is making good progress and will be a
> > great improvement for stateful Kafka applications.
> >
> > Apologies for my late reply, I was away for a while. Lots of great
> comments
> > so far, so I'll probably second most of them in what I suggest below at
> > this point.
> >
> > When I first read the KIP, I wanted to start at the end with something
> that
> > wasn't highlighted a lot. That was the topic related to handling
> duplicate
> > members. I see now that the initial suggestion of handling this situation
> > during offset commit has been removed, and I agree with that. Issues
> > related to membership seem to be handled better when the member joins the
> > group rather than when it tries to commit offsets. This also simplifies
> how
> > many request types need to change in order to incorporate the new member
> > name field.
> >
> > I also agree with what Jason and Guozhang have said regarding timeouts.
> > Although semantically, it's easier to think of every operation having its
> > own timeout, operationally this can become a burden. Thus, consolidation
> > seems preferable here. The definition of embedded protocols on top of the
> > base group membership protocol for rebalancing gives enough flexibility
> to
> > address such needs in each client component separately.
> >
> > Finally, some minor comments:
> > In a few places the new/proposed changes are referred to as "current".
> > Which is a bit confusing considering that there is a protocol in place
> > already, and by "current" someone might understand the existing one. I'd
> > recommend using new/proposed or equivalent when referring to changes
> > introduced with KIP-345 and current/existing or equivalent when referring
> > to existing behavior.
> >
> > There's the following sentence in the "Public Interfaces" section:
> > "Since for many stateful consumer/stream applications, the state
> shuffling
> > is more painful than short time partial unavailability."
> > However, my understanding is that the changes proposed with KIP-345 will
> > not exploit any partial availability. A suggestion for dealing with
> > temporary imbalances has been made in "Incremental Cooperative
> Rebalancing"
> > which can work well with KIP-345, but here I don't see proposed changes
> > that suggest that some resources (e.g. partitions) will keep being used
> > while others will not be utilized. Thus, you might want to adjust this
> > sentence. Correct me if I'm missing something related to that.
> >
> > In the rejected alternatives, under point 2) I read "we can copy the
> member
> > id to the config files". I believe it means to say "member name" unless
> I'm
> > missing something about reusing member ids. Also below I read: "By
> allowing
> > consumers to optionally specifying a member id" which probably implies
> > "member name" again. In a sense this section highlights a potential
> > confusion between member name and member id. I wonder if we could come up
> > with a better term for the new field. StaticTag, StaticLabel, or even
> > StaticName are some suggestions that could potentially help with
> confusion
> > between MemberId and MemberName and what corresponds to what. But I
> > wouldn't like to disrupt the discussion with naming conventions too much
> at
> > this point. I just mention it here as a thought.
> >
> > Looking forward to see the final details of this KIP. Great work so far!
> >
> > Konstantine
> >
> >
> > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Guozhang for the great summary here, and I have been following
> up
> > > the action items here.
> > >
> > >
> > >   1.  I already updated the KIP to remove the expansion timeout and
> > > registration timeout. Great to see them being addressed in client side!
> > >   2.  I double checked the design and I believe that it is ok to have
> > both
> > > static member and dynamic member co-exist in the same group. So the
> > upgrade
> > > shouldn't be destructive and we are removing the two membership
> protocol
> > > switching APIs.
> > >   3.  I only have question about this one. I'm still reading the
> > KafkaApis
> > > code here. Should I just use the same authorization logic for
> > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > >   4.  I'm very excited to see this work with K8! Like you suggested,
> this
> > > feature could be better addressed in a separate KIP because it is
> pretty
> > > independent. I could start drafting the KIP once the current proposal
> is
> > > approved.
> > >   5.  I believe that we don't need fencing in offset commit request,
> > since
> > > duplicate member.name issue could be handled by join group request. We
> > > shall reject join group with known member name but no member id (which
> > > means we already have an active member using this identity).
> > >   6.  I agree to remove that internal config once we move forward with
> > > static membership. And I already removed the entire section from the
> KIP.
> > >
> > > Let me know if you have other concerns.
> > >
> > > Best,
> > > Boyang
> > > ________________________________
> > > From: Guozhang Wang <wa...@gmail.com>
> > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hello Boyang,
> > >
> > > Thanks a lot for the KIP! It is a great write-up and I appreciate your
> > > patience answering to the feedbacks from the community. I'd like to add
> > my
> > > 2cents here:
> > >
> > > 1. By introducing another two timeout configs, registration_timeout and
> > > expansion_timeout, we are effectively having four timeout configs:
> > session
> > > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> > client
> > > side), and these two. Interplaying these timeout configs can be quite
> > hard
> > > for users with such complexity, and hence I'm wondering if we can
> > simplify
> > > the situation with as less possible timeout configs as possible. Here
> is
> > a
> > > concrete suggestion I'd like propose:
> > >
> > > 1.a) Instead of introducing a registration_timeout in addition to the
> > > session_timeout for static members, we can just reuse the
> session_timeout
> > > and ask users to set it to a larger value when they are upgrading a
> > dynamic
> > > client to a static client by setting the "member.name" at the same
> time.
> > > By
> > > default, the broker-side min.session.timeout is 6 seconds and
> > > max.session.timeout is 5 minutes, which seems reasonable to me (we can
> of
> > > course modify this broker config to enlarge the valid interval if we
> want
> > > in practice). And then we should also consider removing the condition
> for
> > > marking a client as failed if the rebalance timeout has reached while
> the
> > > JoinGroup was not received, so that the semantics of session_timeout
> and
> > > rebalance_timeout are totally separated: the former is only used to
> > > determine if a consumer member of the group should be marked as failed
> > and
> > > kicked out of the group, and the latter is only used to determine the
> > > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> > other
> > > words if a member did not send the JoinGroup in time of the
> > > rebalance_timeout, we still include it in the new generation of the
> group
> > > and use its old subscription info to send to leader for assignment.
> Later
> > > if the member came back with HeartBeat request, we can still follow the
> > > normal path to bring it to the latest generation while checking that
> its
> > > sent JoinGroup request contains the same subscription info as we used
> to
> > > assign the partitions previously (which should be likely the case in
> > > practice). In addition, we should let static members to not send the
> > > LeaveGroup request when it is gracefully shutdown, so that a static
> > member
> > > can only be leaving the group if its session has timed out, OR it has
> > been
> > > indicated to not exist in the group any more (details below).
> > >
> > > 1.b) We have a parallel discussion about Incremental Cooperative
> > > Rebalancing, in which we will encode the "when to rebalance" logic at
> the
> > > application level, instead of at the protocol level. By doing this we
> can
> > > also enable a few other optimizations, e.g. at the Streams level to
> first
> > > build up the state store as standby tasks and then trigger a second
> > > rebalance to actually migrate the active tasks while keeping the actual
> > > rebalance latency and hence unavailability window to be small (
> > >
> > >
> >
> https://nam05.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=ZJlsB%2FHLhZykd9mtu5CINNNEqMBvX75bdhqR3IlxGI8%3D&amp;reserved=0
> > ).
> > > I'd propose we align
> > > KIP-345 along with this idea, and hence do not add the
> expansion_timeout
> > as
> > > part of the protocol layer, but only do that at the application's
> > > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > > deprecate the "*group.initial.rebalance.delay.ms
> > > <
> > >
> >
> https://nam05.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cdde139857e7a4a3a83dd08d651d9c93e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636786393153281080&amp;sdata=DDhjV41nPU3euYCQ3w8WPENuw9fPB6ah2j6rF0JjRBg%3D&amp;reserved=0
> > >*"
> > > though as part of this KIP
> > > since we have discussed about its limit and think it is actually not a
> > very
> > > good design and could be replaced with client-side logic above.
> > >
> > >
> > > 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> > > specifically, let's say after we have upgraded broker version to be
> able
> > to
> > > recognize the new versions of JoinGroup request and the admin requests,
> > how
> > > should we upgrade the clients and enable static groups? On top of my
> head
> > > if we do a rolling bounce in which we set the member.name config as
> well
> > > as
> > > optionally increase the session.timeout config when we bounce each
> > > instance, then during this rolling bounces we will have a group
> contained
> > > with both dynamic members and static members. It means that we should
> > have
> > > the group to allow such scenario (i.e. we cannot reject JoinGroup
> > requests
> > > from dynamic members), and hence the "member.name" -> "member.id"
> > mapping
> > > will only be partial at this scenario. Also could you describe if the
> > > upgrade to the first version that support this feature would ever get
> any
> > > benefits, or only the future upgrade path for rolling bounces could get
> > > benefits out of this feature?
> > >
> > > If that's the case and we will do 1) as suggested above, do we still
> need
> > > the enableStaticMembership and enableDynamicMembership admin requests
> any
> > > more? Seems it is not necessary any more as we will only have the
> notion
> > of
> > > "dynamic or static members" that can co-exist in a group while there no
> > > notion of "dynamic or static groups", and hence these two requests are
> > not
> > > needed anymore.
> > >
> > >
> > > 3. We need to briefly talk about the implications for ACL as we
> introduce
> > > new admin requests that are related to a specific group.id. For
> example,
> > > we
> > > need to make sure that whoever created the group or joined the group
> can
> > > actually send admin requests for the group, otherwise the application
> > > owners need to bother the Kafka operators on a multi-tenant cluster
> every
> > > time they want to send any admin requests for their groups which would
> be
> > > an operational nightmare.
> > >
> > >
> > > 4. I like Jason's suggestion of adding an optional field for the list
> of
> > > member names, and I'm wondering if that can be done as part of the
> > > forceStaticRebalance request: i.e. by passing a list of members, we
> will
> > > enforce a rebalance immediately since it indicates that some static
> > member
> > > will be officially kicked out of the group and some new static members
> > may
> > > be added. So back to 1.a) above, a static member can only be kicked out
> > of
> > > the group if a) its session (arguably long period of time) has timed
> out,
> > > and b) this admin request explicitly state that it is no longer part of
> > the
> > > group. As for execution I'm fine with keeping it as a future work of
> this
> > > KIP if you'd like to make its scope smaller.
> > >
> > > Following are minor comments:
> > >
> > > 5. I'm not sure if we need to include "member.name" as part of the
> > > OffsetCommitRequest for fencing purposes, as I think the memberId plus
> > the
> > > generation number should be sufficient for fencing even with static
> > > members.
> > >
> > > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > >
> > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Boyang,
> > > >
> > > > Thanks for the proposal! This is very useful. I have some comments
> > below:
> > > >
> > > > 1) The motivation currently explicitly states that the goal is to
> > improve
> > > > performance for heavy state application. It seems that the motivation
> > can
> > > > be stronger with the following use-case. Currently for MirrorMaker
> > > cluster
> > > > with e.g. 100 MirrorMaker processes, it will take a long time to
> > rolling
> > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> restart
> > > > will trigger a rebalance which currently pause the consumption of the
> > all
> > > > partitions of the MirrorMaker cluster. With the change stated in this
> > > > patch, as long as a MirrorMaker can restart within the specified
> > timeout
> > > > (e.g. 2 minutes), then we only need constant number of rebalance
> (e.g.
> > > for
> > > > leader restart) for the entire rolling bounce, which will
> significantly
> > > > improves the availability of the MirrorMaker pipeline. In my opinion,
> > the
> > > > main benefit of the KIP is to avoid unnecessary rebalance if the
> > consumer
> > > > process can be restarted within soon, which helps performance even if
> > > > overhead of state shuffling for a given process is small.
> > > >
> > > > 2) In order to simplify the KIP reading, can you follow the writeup
> > style
> > > > of other KIP (e.g. KIP-98) and list the interface change such as new
> > > > configs (e.g. registration timeout), new request/response, new
> > > AdminClient
> > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> > of
> > > > these are specified in the Proposed Change section which makes it a
> bit
> > > > inconvenient to understand the new interface that will be exposed to
> > > user.
> > > > Explanation of the current two-phase rebalance protocol probably can
> be
> > > > moved out of public interface section.
> > > >
> > > > 3) There are currently two version of JoinGroupRequest in the KIP and
> > > only
> > > > one of them has field memberId. This seems confusing.
> > > >
> > > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> > could
> > > > be helpful here, but we will make a call once we finished the major
> > > > implementation". So this seems to be still an open question in the
> > > current
> > > > design. We probably want to agree on this before voting for the KIP.
> > > >
> > > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> you
> > > > specify the name of the config key and the default config value?
> > Possible
> > > > default values include empty string or null (similar to
> transaction.id
> > > in
> > > > producer config).
> > > >
> > > > 6) Regarding the use of the topic "static_member_map" to persist
> member
> > > > name map, currently if consumer coordinator broker goes offline,
> > > rebalance
> > > > is triggered and consumers will try connect to the new coordinator.
> If
> > > > these consumers can connect to the new coordinator within
> > > > max.poll.interval.ms which by default is 5 minutes, given that
> broker
> > > can
> > > > use a deterministic algorithm to determine the partition ->
> member_name
> > > > mapping, each consumer should get assigned the same set of partitions
> > > > without requiring state shuffling. So it is not clear whether we
> have a
> > > > strong use-case for this new logic. Can you help clarify what is the
> > > > benefit of using topic "static_member_map" to persist member name
> map?
> > > >
> > > > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > > > mentioned that "we are using expansion timeout to replace rebalance
> > > > timeout, which is configured by max.poll.intervals from client side,
> > and
> > > > using registration timeout to replace session timeout". Currently the
> > > > default max.poll.interval.ms is configured to be 5 minutes and there
> > > will
> > > > be only one rebalance if all new consumers can join within 5 minutes.
> > So
> > > it
> > > > is not clear whether we have a strong use-case for this new config.
> Can
> > > you
> > > > explain what is the benefit of introducing this new config?
> > > >
> > > > 8) It is mentioned that "To distinguish between previous version of
> > > > protocol, we will also increase the join group request version to v4
> > when
> > > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> > v4),
> > > > the join group request shall be downgraded to v3 without setting the
> > > member
> > > > Id". It is probably simpler to just say that this feature is enabled
> if
> > > > JoinGroupRequest V4 is supported on both client and broker and
> > > MEMBER_NAME
> > > > is configured with non-empty string.
> > > >
> > > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> > error
> > > > in OffsetCommitResponse for "commit requests under static
> membership".
> > > Can
> > > > you clarify how broker determines whether the commit request is under
> > > > static membership?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Dong Lin <li...@gmail.com>.
Hey Boyang,

Thanks for the update! Here are some followup comments:

1) It is not very clear to the user what is the difference between
member.name and client.id as both seems to be used to identify the
consumer. I am wondering if it would be more intuitive to name it
group.member.name (preferred choice since it matches the current group.id
config name) or rebalance.member.name to explicitly show that the id is
solely used for rebalance.

2) In the interface change section it is said that GroupMaxSessionTimeoutMs
will be changed to 30 minutes. It seems to suggest that we will change the
default value of this config. It does not seem necessary to increase the
time of consumer failure detection when user doesn't use static membership.
Also, say static membership is enabled, then this default config change
will cause a partition to be unavailable for consumption for 30 minutes if
there is hard consumer failure, which seems to be worse experience than
having unnecessary rebalance (when this timeout is small), particularly for
new users of Kafka. Could you explain more why we should make this change?

3) Could we just combine MEMBER_ID_MISMATCH and DUPLICATE_STATIC_MEMBER
into one error? It seems that these two errors are currently handled by the
consumer in the same way. And we don't also don't expect MEMBER_ID_MISMATCH
to happen. Thus it is not clear what is the benefit of having two errors.

4) The doc for DUPLICATE_STATIC_MEMBER says that "The join group contains
member name which is already in the consumer group, however the member id
was missing". After a consumer is restarted, it will send a
JoinGroupRequest with an existing memberName (as the coordinator has not
expired this member from the memory) and memberId
= JoinGroupRequest.UNKNOWN_MEMBER_ID (since memberId is not persisted
across consumer restart in the consumer side). Does it mean that
JoinGroupRequest from a newly restarted consumer will always be rejected
until the sessionTimeoutMs has passed?

5) It seems that we always add two methods to the interface
org.apache.kafka.clients.admin.AdminClient.java, one with options and the
other without option. Could this be specified in the interface change
section?

6) Do we plan to have off-the-shelf command line tool for SRE to trigger
rebalance? If so, we probably want to specify the command line tool
interface similar to
https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories#KIP-113:Supportreplicasmovementbetweenlogdirectories-Scripts
.

7) Would it be simpler to replace name "forceStaticRebalance" with
"invokeConsumerRebalance"? It is not very clear what is the extra meaning
of world "force" as compared to "trigger" or "invoke". And it seems simpler
to allows this API to trigger rebalance regardless of whether consumer is
configured with memberName.

8) It is not very clear how the newly added AdminClient API trigger
rebalance. For example, does it send request? Can this be explained in the
KIP?

Thanks,
Dong


On Thu, Nov 22, 2018 at 6:37 AM Boyang Chen <bc...@outlook.com> wrote:

> Hey Mayuresh,
>
>
> thanks for your feedbacks! I will try do another checklist here.
>
>
> > By this you mean, even if the application has not called
> > KafkaConsumer.poll() within session timeout, it will not be sending the
> > LeaveGroup request, right?
>
> Yep it's true, we will prevent client from sending leave group request
> when they are set with `member.name`.
>
>
> > When is the member.name removed from this map?
> Good question, we will only kick off member due to session timeout within
> static membership. Let me update the KIP to clearly assert that.
>
> > How is this case (missing member id) handled on the client side? What is
> the application that
> > is using the KafkaConsumer suppose to do in this scenario?
> I have extended the two exceptions within join group response V4.
> Basically I define both corresponding actions to be immediate failing
> client application, because so far it is unknown what kind of client issue
> could trigger them. After the first version, we will keep enhance the error
> handling logic!
>
> > This would mean that it might take more time to detect unowned topic
> > partitions and may cause delay for applications that perform data
> mirroring
> > tasks. I discussed this with our sre and we have a suggestion to make
> here
> > as listed below separately.
> The goal of extending session timeout cap is for users with good client
> side monitoring tools that could auto-heal the dead consumers very fast. So
> it is optional (and personal) to extend session timeout to a reasonable
> number with different client scenarios.
>
> > you meant remove unjoined members of the group, right ?
> Yep, there is a typo. Thanks for catching this!
>
> > What do you mean by " Internally we would optimize this logic by having
> > rebalance timeout only in charge of stopping prepare rebalance stage,
> > without removing non-responsive members immediately." There would not be
> a
> > full rebalance if the lagging consumer sent a JoinGroup request later,
> > right ? If yes, can you highlight this in the KIP ?
> No, there won't be. We want to limit the rebalance timeout functionality
> to only use as a timer to
> end prepare rebalance stage. This way, late joining static members will
> not trigger further rebalance
> as long as they are within session timeout. I added your highlight to the
> KIP!
>
> > The KIP talks about scale up scenario but its not quite clear how we
> > handle it. Are we adding a separate "expansion.timeout" or we adding
> status
> > "learner" ?. Can you shed more light on how this is handled in the KIP,
> if
> > its handled?
> Updated the KIP: we shall not cover scale up case in 345, because we
> believe client side could
> better handle this logic.
>
> > I think Jason had brought this up earlier about having a way to say how
> > many members/consumer hosts are you choosing to be in the consumer group.
> > If we can do this, then in case of mirroring applications we can do this
> :
> > Lets say we have a mirroring application that consumes from Kafka cluster
> > A and produces to Kafka cluster B.
> > Depending on the data and the Kafka cluster configuration, Kafka service
> > providers can set a mirroring group saying that it will take, for example
> > 300 consumer hosts/members to achieve the desired throughput and latency
> > for mirroring and can have additional 10 consumer hosts as spare in the
> > same group.
> > So when the first 300 members/consumers to join the group will start
> > mirroring the data from Kafka cluster A to Kafka cluster B.
> > The remaining 10 consumer members can sit idle.
> > The moment one of the consumer (for example: consumer number 54) from the
> > first 300 members go out of the group (crossed session timeout), it (the
> > groupCoordinator) can just assign the topicPartitions from the consumer
> > member 54 to one of the spare hosts.
> > Once the consumer member 54 comes back up, it can start as being a part
> of
> > the spare pool.
> > This enables us to have lower session timeouts and low latency mirroring,
> > in cases where the service providers are OK with having spare hosts.
> > This would mean that we would tolerate n consumer members leaving and
> > rejoining the group and still provide low latency as long as n <= number
> of
> > spare consumers.
> > If there are no spare host available, we can get back to the idea as
> > described in the KIP.
> Great idea! In fact on top of static membership we could later introduce
> APIs to set hard-coded
> client ids to the group and replace the dead host, or as you proposed to
> define spare host as
> what I understood as hot backup. I will put both Jason and your
> suggestions into a separate section
> called "Future works". Note that this spare host idea may be also solvable
> through rebalance protocol
> IMO.
>
> Thank you again for the great feedback!
>
> Boyang
> ________________________________
> From: Boyang Chen <bc...@outlook.com>
> Sent: Thursday, November 22, 2018 3:39 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey Dong, sorry for missing your message. I couldn't find your email on my
> thread, so I will just do a checklist here!
>
>
> 1) The motivation currently explicitly states that the goal is to improve
>
> performance for heavy state application. It seems that the motivation can
>
> be stronger with the following use-case. Currently for MirrorMaker cluster
>
> with e.g. 100 MirrorMaker processes, it will take a long time to rolling
>
> bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
>
> will trigger a rebalance which currently pause the consumption of the all
>
> partitions of the MirrorMaker cluster. With the change stated in this
>
> patch, as long as a MirrorMaker can restart within the specified timeout
>
> (e.g. 2 minutes), then we only need constant number of rebalance (e.g. for
>
> leader restart) for the entire rolling bounce, which will significantly
>
> improves the availability of the MirrorMaker pipeline. In my opinion, the
>
> main benefit of the KIP is to avoid unnecessary rebalance if the consumer
>
> process can be restarted within soon, which helps performance even if
>
> overhead of state shuffling for a given process is small.
>
> I just rephrased this part and added it to the KIP. Thanks for making the
> motivation more solid!
>
> 2) In order to simplify the KIP reading, can you follow the writeup style
> of other KIP (e.g. KIP-98) and list the interface change such as new
> configs (e.g. registration timeout), new request/response, new AdminClient
> API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> these are specified in the Proposed Change section which makes it a bit
> inconvenient to understand the new interface that will be exposed to user.
> Explanation of the current two-phase rebalance protocol probably can be
> moved out of public interface section.
> This is a great suggestion! I just consolidated all the public API
> changes, and the whole KIP
> looks much more organized!
>
> 3) There are currently two version of JoinGroupRequest in the KIP and only
> one of them has field memberId. This seems confusing.
> Yep, I already found this issue and fixed it.
>
> 4) It is mentioned in the KIP that "An admin API to force rebalance could
> be helpful here, but we will make a call once we finished the major
> implementation". So this seems to be still an open question in the current
> design. We probably want to agree on this before voting for the KIP.
> We have finalized the idea that this API is needed.
>
> 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> specify the name of the config key and the default config value? Possible
> default values include empty string or null (similar to transaction.id<
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=sA3XnqHD2s8ya93rOBg5IZvJmbkUqliBGQ3ouyOAUFk%3D&amp;reserved=0>
> in
> producer config).
> I have defined the `member.name` in "New configuration" section.
>
> 6) Regarding the use of the topic "static_member_map" to persist member
> name map, currently if consumer coordinator broker goes offline, rebalance
> is triggered and consumers will try connect to the new coordinator. If
> these consumers can connect to the new coordinator within
> max.poll.interval.ms<
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=INHjX2K9iIfEBC9eUvuxLWuiELaE1uSfr4BmxWyfusg%3D&amp;reserved=0>
> which by default is 5 minutes, given that broker can
> use a deterministic algorithm to determine the partition -> member_name
> mapping, each consumer should get assigned the same set of partitions
> without requiring state shuffling. So it is not clear whether we have a
> strong use-case for this new logic. Can you help clarify what is the
> benefit of using topic "static_member_map" to persist member name map?
> I have discussed with Guozhang offline, and I believe reusing the current
> `_consumer_offsets`
> topic is a better and unified solution.
>
> 7) Regarding the introduction of the expensionTimeoutMs config, it is
> mentioned that "we are using expansion timeout to replace rebalance
> timeout, which is configured by max.poll.intervals from client side, and
> using registration timeout to replace session timeout". Currently the
> default max.poll.interval.ms<
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=INHjX2K9iIfEBC9eUvuxLWuiELaE1uSfr4BmxWyfusg%3D&amp;reserved=0>
> is configured to be 5 minutes and there will
> be only one rebalance if all new consumers can join within 5 minutes. So it
> is not clear whether we have a strong use-case for this new config. Can you
> explain what is the benefit of introducing this new config?
> Previously our goal is to use expansion timeout as a workaround for
> triggering multiple
> rebalances when scaling up members are not joining at the same time. It is
> decided to
> be addressed by client side protocol change, so we will not introduce
> expansion timeout.
>
> 8) It is mentioned that "To distinguish between previous version of
> protocol, we will also increase the join group request version to v4 when
> MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> the join group request shall be downgraded to v3 without setting the member
> Id". It is probably simpler to just say that this feature is enabled if
> JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
> is configured with non-empty string.
> Yep, addressed this!
>
> 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> in OffsetCommitResponse for "commit requests under static membership". Can
> you clarify how broker determines whether the commit request is under
> static membership?
>
> We have agreed that commit request shouldn't be affected by the new
> membership, thus
> removing it here. Thanks for catching this!
>
> Let me know if you have further suggestions or concerns. Thank you for
> your valuable feedback
> to help me design the KIP better! (And I will try to address your
> feedbacks in next round Mayuresh 😊)
>
> Best,
> Boyang
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Wednesday, November 21, 2018 7:50 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for updating the KIP. This is a step good direction for stateful
> applications and also mirroring applications whose latency is affected due
> to the rebalance issues that we have today.
>
> I had a few questions on the current version of the KIP :
> For the effectiveness of the KIP, consumer with member.name set will *not
> send leave group request* when they go offline
>
> > By this you mean, even if the application has not called
> > KafkaConsumer.poll() within session timeout, it will not be sending the
> > LeaveGroup request, right?
> >
>
> Broker will maintain an in-memory mapping of {member.name → member.id} to
> track member uniqueness.
>
> > When is the member.name removed from this map?
> >
>
> Member.id must be set if the *member.name <
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=C7pxl8FcLQamaFFJQ88OhUN7ATIa8GimdJDhSq6gxug%3D&amp;reserved=0>
> *is already
> within the map. Otherwise reply MISSING_MEMBER_ID
>
> > How is this case handled on the client side? What is the application that
> > is using the KafkaConsumer suppose to do in this scenario?
> >
>
> Session timeout is the timeout we will trigger rebalance when a member goes
> offline for too long (not sending heartbeat request). To make static
> membership effective, we should increase the default max session timeout to
> 30 min so that end user could config it freely.
>
> > This would mean that it might take more time to detect unowned topic
> > partitions and may cause delay for applications that perform data
> mirroring
> > tasks. I discussed this with our sre and we have a suggestion to make
> here
> > as listed below separately.
> >
>
> Currently there is a config called *rebalance timeout* which is configured
> by consumer *max.poll.intervals*. The reason we set it to poll interval is
> because consumer could only send request within the call of poll() and we
> want to wait sufficient time for the join group request. When reaching
> rebalance timeout, the group will move towards completingRebalance stage
> and remove unjoined groups
>
> > you meant remove unjoined members of the group, right ?
> >
>
> Currently there is a config called *rebalance timeout* which is configured
> by consumer *max.poll.intervals*. The reason we set it to poll interval is
> because consumer could only send request within the call of poll() and we
> want to wait sufficient time for the join group request. When reaching
> rebalance timeout, the group will move towards completingRebalance stage
> and remove unjoined groups. This is actually conflicting with the design of
> static membership, because those temporarily unavailable members will
> potentially reattempt the join group and trigger extra rebalances.
> Internally we would optimize this logic by having rebalance timeout only in
> charge of stopping prepare rebalance stage, without removing non-responsive
> members immediately.
>
> > What do you mean by " Internally we would optimize this logic by having
> > rebalance timeout only in charge of stopping prepare rebalance stage,
> > without removing non-responsive members immediately." There would not be
> a
> > full rebalance if the lagging consumer sent a JoinGroup request later,
> > right ? If yes, can you highlight this in the KIP ?
> >
>
> Scale Up
>
> > The KIP talks about scale up scenario but its not quite clear how we
> > handle it. Are we adding a separate "expansion.timeout" or we adding
> status
> > "learner" ?. Can you shed more light on how this is handled in the KIP,
> if
> > its handled?
> >
>
>
> *Discussion*
> Larger session timeouts causing latency rise for getting data for un-owned
> topic partitions :
>
> > I think Jason had brought this up earlier about having a way to say how
> > many members/consumer hosts are you choosing to be in the consumer group.
> > If we can do this, then in case of mirroring applications we can do this
> :
> > Lets say we have a mirroring application that consumes from Kafka cluster
> > A and produces to Kafka cluster B.
> > Depending on the data and the Kafka cluster configuration, Kafka service
> > providers can set a mirroring group saying that it will take, for example
> > 300 consumer hosts/members to achieve the desired throughput and latency
> > for mirroring and can have additional 10 consumer hosts as spare in the
> > same group.
> > So when the first 300 members/consumers to join the group will start
> > mirroring the data from Kafka cluster A to Kafka cluster B.
> > The remaining 10 consumer members can sit idle.
> > The moment one of the consumer (for example: consumer number 54) from the
> > first 300 members go out of the group (crossed session timeout), it (the
> > groupCoordinator) can just assign the topicPartitions from the consumer
> > member 54 to one of the spare hosts.
> > Once the consumer member 54 comes back up, it can start as being a part
> of
> > the spare pool.
> > This enables us to have lower session timeouts and low latency mirroring,
> > in cases where the service providers are OK with having spare hosts.
> > This would mean that we would tolerate n consumer members leaving and
> > rejoining the group and still provide low latency as long as n <= number
> of
> > spare consumers.
> > If there are no spare host available, we can get back to the idea as
> > described in the KIP.
> >
>
> Thanks,
>
> Mayuresh
>
>
>
>
>
> On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
> konstantine@confluent.io> wrote:
>
> > Hi Boyang.
> >
> > Thanks for preparing this KIP! It is making good progress and will be a
> > great improvement for stateful Kafka applications.
> >
> > Apologies for my late reply, I was away for a while. Lots of great
> comments
> > so far, so I'll probably second most of them in what I suggest below at
> > this point.
> >
> > When I first read the KIP, I wanted to start at the end with something
> that
> > wasn't highlighted a lot. That was the topic related to handling
> duplicate
> > members. I see now that the initial suggestion of handling this situation
> > during offset commit has been removed, and I agree with that. Issues
> > related to membership seem to be handled better when the member joins the
> > group rather than when it tries to commit offsets. This also simplifies
> how
> > many request types need to change in order to incorporate the new member
> > name field.
> >
> > I also agree with what Jason and Guozhang have said regarding timeouts.
> > Although semantically, it's easier to think of every operation having its
> > own timeout, operationally this can become a burden. Thus, consolidation
> > seems preferable here. The definition of embedded protocols on top of the
> > base group membership protocol for rebalancing gives enough flexibility
> to
> > address such needs in each client component separately.
> >
> > Finally, some minor comments:
> > In a few places the new/proposed changes are referred to as "current".
> > Which is a bit confusing considering that there is a protocol in place
> > already, and by "current" someone might understand the existing one. I'd
> > recommend using new/proposed or equivalent when referring to changes
> > introduced with KIP-345 and current/existing or equivalent when referring
> > to existing behavior.
> >
> > There's the following sentence in the "Public Interfaces" section:
> > "Since for many stateful consumer/stream applications, the state
> shuffling
> > is more painful than short time partial unavailability."
> > However, my understanding is that the changes proposed with KIP-345 will
> > not exploit any partial availability. A suggestion for dealing with
> > temporary imbalances has been made in "Incremental Cooperative
> Rebalancing"
> > which can work well with KIP-345, but here I don't see proposed changes
> > that suggest that some resources (e.g. partitions) will keep being used
> > while others will not be utilized. Thus, you might want to adjust this
> > sentence. Correct me if I'm missing something related to that.
> >
> > In the rejected alternatives, under point 2) I read "we can copy the
> member
> > id to the config files". I believe it means to say "member name" unless
> I'm
> > missing something about reusing member ids. Also below I read: "By
> allowing
> > consumers to optionally specifying a member id" which probably implies
> > "member name" again. In a sense this section highlights a potential
> > confusion between member name and member id. I wonder if we could come up
> > with a better term for the new field. StaticTag, StaticLabel, or even
> > StaticName are some suggestions that could potentially help with
> confusion
> > between MemberId and MemberName and what corresponds to what. But I
> > wouldn't like to disrupt the discussion with naming conventions too much
> at
> > this point. I just mention it here as a thought.
> >
> > Looking forward to see the final details of this KIP. Great work so far!
> >
> > Konstantine
> >
> >
> > On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Guozhang for the great summary here, and I have been following
> up
> > > the action items here.
> > >
> > >
> > >   1.  I already updated the KIP to remove the expansion timeout and
> > > registration timeout. Great to see them being addressed in client side!
> > >   2.  I double checked the design and I believe that it is ok to have
> > both
> > > static member and dynamic member co-exist in the same group. So the
> > upgrade
> > > shouldn't be destructive and we are removing the two membership
> protocol
> > > switching APIs.
> > >   3.  I only have question about this one. I'm still reading the
> > KafkaApis
> > > code here. Should I just use the same authorization logic for
> > > ForceStaticRebalanceRequest as JoinGroupRequest?
> > >   4.  I'm very excited to see this work with K8! Like you suggested,
> this
> > > feature could be better addressed in a separate KIP because it is
> pretty
> > > independent. I could start drafting the KIP once the current proposal
> is
> > > approved.
> > >   5.  I believe that we don't need fencing in offset commit request,
> > since
> > > duplicate member.name issue could be handled by join group request. We
> > > shall reject join group with known member name but no member id (which
> > > means we already have an active member using this identity).
> > >   6.  I agree to remove that internal config once we move forward with
> > > static membership. And I already removed the entire section from the
> KIP.
> > >
> > > Let me know if you have other concerns.
> > >
> > > Best,
> > > Boyang
> > > ________________________________
> > > From: Guozhang Wang <wa...@gmail.com>
> > > Sent: Tuesday, November 20, 2018 4:21 PM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hello Boyang,
> > >
> > > Thanks a lot for the KIP! It is a great write-up and I appreciate your
> > > patience answering to the feedbacks from the community. I'd like to add
> > my
> > > 2cents here:
> > >
> > > 1. By introducing another two timeout configs, registration_timeout and
> > > expansion_timeout, we are effectively having four timeout configs:
> > session
> > > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> > client
> > > side), and these two. Interplaying these timeout configs can be quite
> > hard
> > > for users with such complexity, and hence I'm wondering if we can
> > simplify
> > > the situation with as less possible timeout configs as possible. Here
> is
> > a
> > > concrete suggestion I'd like propose:
> > >
> > > 1.a) Instead of introducing a registration_timeout in addition to the
> > > session_timeout for static members, we can just reuse the
> session_timeout
> > > and ask users to set it to a larger value when they are upgrading a
> > dynamic
> > > client to a static client by setting the "member.name" at the same
> time.
> > > By
> > > default, the broker-side min.session.timeout is 6 seconds and
> > > max.session.timeout is 5 minutes, which seems reasonable to me (we can
> of
> > > course modify this broker config to enlarge the valid interval if we
> want
> > > in practice). And then we should also consider removing the condition
> for
> > > marking a client as failed if the rebalance timeout has reached while
> the
> > > JoinGroup was not received, so that the semantics of session_timeout
> and
> > > rebalance_timeout are totally separated: the former is only used to
> > > determine if a consumer member of the group should be marked as failed
> > and
> > > kicked out of the group, and the latter is only used to determine the
> > > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> > other
> > > words if a member did not send the JoinGroup in time of the
> > > rebalance_timeout, we still include it in the new generation of the
> group
> > > and use its old subscription info to send to leader for assignment.
> Later
> > > if the member came back with HeartBeat request, we can still follow the
> > > normal path to bring it to the latest generation while checking that
> its
> > > sent JoinGroup request contains the same subscription info as we used
> to
> > > assign the partitions previously (which should be likely the case in
> > > practice). In addition, we should let static members to not send the
> > > LeaveGroup request when it is gracefully shutdown, so that a static
> > member
> > > can only be leaving the group if its session has timed out, OR it has
> > been
> > > indicated to not exist in the group any more (details below).
> > >
> > > 1.b) We have a parallel discussion about Incremental Cooperative
> > > Rebalancing, in which we will encode the "when to rebalance" logic at
> the
> > > application level, instead of at the protocol level. By doing this we
> can
> > > also enable a few other optimizations, e.g. at the Streams level to
> first
> > > build up the state store as standby tasks and then trigger a second
> > > rebalance to actually migrate the active tasks while keeping the actual
> > > rebalance latency and hence unavailability window to be small (
> > >
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=wrqKK%2BWMoA%2BF0fY%2BergbA73YTV6twD8lpcz1RYOvMKw%3D&amp;reserved=0
> > ).
> > > I'd propose we align
> > > KIP-345 along with this idea, and hence do not add the
> expansion_timeout
> > as
> > > part of the protocol layer, but only do that at the application's
> > > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > > deprecate the "*group.initial.rebalance.delay.ms
> > > <
> > >
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=q7SAR3tHJoFW0qZwwOdGk9FFNMIbUzPJfhR9iPuXPOM%3D&amp;reserved=0
> > >*"
> > > though as part of this KIP
> > > since we have discussed about its limit and think it is actually not a
> > very
> > > good design and could be replaced with client-side logic above.
> > >
> > >
> > > 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> > > specifically, let's say after we have upgraded broker version to be
> able
> > to
> > > recognize the new versions of JoinGroup request and the admin requests,
> > how
> > > should we upgrade the clients and enable static groups? On top of my
> head
> > > if we do a rolling bounce in which we set the member.name config as
> well
> > > as
> > > optionally increase the session.timeout config when we bounce each
> > > instance, then during this rolling bounces we will have a group
> contained
> > > with both dynamic members and static members. It means that we should
> > have
> > > the group to allow such scenario (i.e. we cannot reject JoinGroup
> > requests
> > > from dynamic members), and hence the "member.name" -> "member.id"
> > mapping
> > > will only be partial at this scenario. Also could you describe if the
> > > upgrade to the first version that support this feature would ever get
> any
> > > benefits, or only the future upgrade path for rolling bounces could get
> > > benefits out of this feature?
> > >
> > > If that's the case and we will do 1) as suggested above, do we still
> need
> > > the enableStaticMembership and enableDynamicMembership admin requests
> any
> > > more? Seems it is not necessary any more as we will only have the
> notion
> > of
> > > "dynamic or static members" that can co-exist in a group while there no
> > > notion of "dynamic or static groups", and hence these two requests are
> > not
> > > needed anymore.
> > >
> > >
> > > 3. We need to briefly talk about the implications for ACL as we
> introduce
> > > new admin requests that are related to a specific group.id. For
> example,
> > > we
> > > need to make sure that whoever created the group or joined the group
> can
> > > actually send admin requests for the group, otherwise the application
> > > owners need to bother the Kafka operators on a multi-tenant cluster
> every
> > > time they want to send any admin requests for their groups which would
> be
> > > an operational nightmare.
> > >
> > >
> > > 4. I like Jason's suggestion of adding an optional field for the list
> of
> > > member names, and I'm wondering if that can be done as part of the
> > > forceStaticRebalance request: i.e. by passing a list of members, we
> will
> > > enforce a rebalance immediately since it indicates that some static
> > member
> > > will be officially kicked out of the group and some new static members
> > may
> > > be added. So back to 1.a) above, a static member can only be kicked out
> > of
> > > the group if a) its session (arguably long period of time) has timed
> out,
> > > and b) this admin request explicitly state that it is no longer part of
> > the
> > > group. As for execution I'm fine with keeping it as a future work of
> this
> > > KIP if you'd like to make its scope smaller.
> > >
> > > Following are minor comments:
> > >
> > > 5. I'm not sure if we need to include "member.name" as part of the
> > > OffsetCommitRequest for fencing purposes, as I think the memberId plus
> > the
> > > generation number should be sufficient for fencing even with static
> > > members.
> > >
> > > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> > >
> > >
> > > Guozhang
> > >
> > >
> > >
> > >
> > > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
> > >
> > > > Hey Boyang,
> > > >
> > > > Thanks for the proposal! This is very useful. I have some comments
> > below:
> > > >
> > > > 1) The motivation currently explicitly states that the goal is to
> > improve
> > > > performance for heavy state application. It seems that the motivation
> > can
> > > > be stronger with the following use-case. Currently for MirrorMaker
> > > cluster
> > > > with e.g. 100 MirrorMaker processes, it will take a long time to
> > rolling
> > > > bounce the entire MirrorMaker cluster. Each MirrorMaker process
> restart
> > > > will trigger a rebalance which currently pause the consumption of the
> > all
> > > > partitions of the MirrorMaker cluster. With the change stated in this
> > > > patch, as long as a MirrorMaker can restart within the specified
> > timeout
> > > > (e.g. 2 minutes), then we only need constant number of rebalance
> (e.g.
> > > for
> > > > leader restart) for the entire rolling bounce, which will
> significantly
> > > > improves the availability of the MirrorMaker pipeline. In my opinion,
> > the
> > > > main benefit of the KIP is to avoid unnecessary rebalance if the
> > consumer
> > > > process can be restarted within soon, which helps performance even if
> > > > overhead of state shuffling for a given process is small.
> > > >
> > > > 2) In order to simplify the KIP reading, can you follow the writeup
> > style
> > > > of other KIP (e.g. KIP-98) and list the interface change such as new
> > > > configs (e.g. registration timeout), new request/response, new
> > > AdminClient
> > > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> > of
> > > > these are specified in the Proposed Change section which makes it a
> bit
> > > > inconvenient to understand the new interface that will be exposed to
> > > user.
> > > > Explanation of the current two-phase rebalance protocol probably can
> be
> > > > moved out of public interface section.
> > > >
> > > > 3) There are currently two version of JoinGroupRequest in the KIP and
> > > only
> > > > one of them has field memberId. This seems confusing.
> > > >
> > > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> > could
> > > > be helpful here, but we will make a call once we finished the major
> > > > implementation". So this seems to be still an open question in the
> > > current
> > > > design. We probably want to agree on this before voting for the KIP.
> > > >
> > > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can
> you
> > > > specify the name of the config key and the default config value?
> > Possible
> > > > default values include empty string or null (similar to
> transaction.id
> > > in
> > > > producer config).
> > > >
> > > > 6) Regarding the use of the topic "static_member_map" to persist
> member
> > > > name map, currently if consumer coordinator broker goes offline,
> > > rebalance
> > > > is triggered and consumers will try connect to the new coordinator.
> If
> > > > these consumers can connect to the new coordinator within
> > > > max.poll.interval.ms which by default is 5 minutes, given that
> broker
> > > can
> > > > use a deterministic algorithm to determine the partition ->
> member_name
> > > > mapping, each consumer should get assigned the same set of partitions
> > > > without requiring state shuffling. So it is not clear whether we
> have a
> > > > strong use-case for this new logic. Can you help clarify what is the
> > > > benefit of using topic "static_member_map" to persist member name
> map?
> > > >
> > > > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > > > mentioned that "we are using expansion timeout to replace rebalance
> > > > timeout, which is configured by max.poll.intervals from client side,
> > and
> > > > using registration timeout to replace session timeout". Currently the
> > > > default max.poll.interval.ms is configured to be 5 minutes and there
> > > will
> > > > be only one rebalance if all new consumers can join within 5 minutes.
> > So
> > > it
> > > > is not clear whether we have a strong use-case for this new config.
> Can
> > > you
> > > > explain what is the benefit of introducing this new config?
> > > >
> > > > 8) It is mentioned that "To distinguish between previous version of
> > > > protocol, we will also increase the join group request version to v4
> > when
> > > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> > v4),
> > > > the join group request shall be downgraded to v3 without setting the
> > > member
> > > > Id". It is probably simpler to just say that this feature is enabled
> if
> > > > JoinGroupRequest V4 is supported on both client and broker and
> > > MEMBER_NAME
> > > > is configured with non-empty string.
> > > >
> > > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> > error
> > > > in OffsetCommitResponse for "commit requests under static
> membership".
> > > Can
> > > > you clarify how broker determines whether the commit request is under
> > > > static membership?
> > > >
> > > > Thanks,
> > > > Dong
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Hey Mayuresh,


thanks for your feedbacks! I will try do another checklist here.


> By this you mean, even if the application has not called
> KafkaConsumer.poll() within session timeout, it will not be sending the
> LeaveGroup request, right?

Yep it's true, we will prevent client from sending leave group request when they are set with `member.name`.


> When is the member.name removed from this map?
Good question, we will only kick off member due to session timeout within static membership. Let me update the KIP to clearly assert that.

> How is this case (missing member id) handled on the client side? What is the application that
> is using the KafkaConsumer suppose to do in this scenario?
I have extended the two exceptions within join group response V4. Basically I define both corresponding actions to be immediate failing client application, because so far it is unknown what kind of client issue could trigger them. After the first version, we will keep enhance the error handling logic!

> This would mean that it might take more time to detect unowned topic
> partitions and may cause delay for applications that perform data mirroring
> tasks. I discussed this with our sre and we have a suggestion to make here
> as listed below separately.
The goal of extending session timeout cap is for users with good client side monitoring tools that could auto-heal the dead consumers very fast. So it is optional (and personal) to extend session timeout to a reasonable number with different client scenarios.

> you meant remove unjoined members of the group, right ?
Yep, there is a typo. Thanks for catching this!

> What do you mean by " Internally we would optimize this logic by having
> rebalance timeout only in charge of stopping prepare rebalance stage,
> without removing non-responsive members immediately." There would not be a
> full rebalance if the lagging consumer sent a JoinGroup request later,
> right ? If yes, can you highlight this in the KIP ?
No, there won't be. We want to limit the rebalance timeout functionality to only use as a timer to
end prepare rebalance stage. This way, late joining static members will not trigger further rebalance
as long as they are within session timeout. I added your highlight to the KIP!

> The KIP talks about scale up scenario but its not quite clear how we
> handle it. Are we adding a separate "expansion.timeout" or we adding status
> "learner" ?. Can you shed more light on how this is handled in the KIP, if
> its handled?
Updated the KIP: we shall not cover scale up case in 345, because we believe client side could
better handle this logic.

> I think Jason had brought this up earlier about having a way to say how
> many members/consumer hosts are you choosing to be in the consumer group.
> If we can do this, then in case of mirroring applications we can do this :
> Lets say we have a mirroring application that consumes from Kafka cluster
> A and produces to Kafka cluster B.
> Depending on the data and the Kafka cluster configuration, Kafka service
> providers can set a mirroring group saying that it will take, for example
> 300 consumer hosts/members to achieve the desired throughput and latency
> for mirroring and can have additional 10 consumer hosts as spare in the
> same group.
> So when the first 300 members/consumers to join the group will start
> mirroring the data from Kafka cluster A to Kafka cluster B.
> The remaining 10 consumer members can sit idle.
> The moment one of the consumer (for example: consumer number 54) from the
> first 300 members go out of the group (crossed session timeout), it (the
> groupCoordinator) can just assign the topicPartitions from the consumer
> member 54 to one of the spare hosts.
> Once the consumer member 54 comes back up, it can start as being a part of
> the spare pool.
> This enables us to have lower session timeouts and low latency mirroring,
> in cases where the service providers are OK with having spare hosts.
> This would mean that we would tolerate n consumer members leaving and
> rejoining the group and still provide low latency as long as n <= number of
> spare consumers.
> If there are no spare host available, we can get back to the idea as
> described in the KIP.
Great idea! In fact on top of static membership we could later introduce APIs to set hard-coded
client ids to the group and replace the dead host, or as you proposed to define spare host as
what I understood as hot backup. I will put both Jason and your suggestions into a separate section
called "Future works". Note that this spare host idea may be also solvable through rebalance protocol
IMO.

Thank you again for the great feedback!

Boyang
________________________________
From: Boyang Chen <bc...@outlook.com>
Sent: Thursday, November 22, 2018 3:39 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hey Dong, sorry for missing your message. I couldn't find your email on my thread, so I will just do a checklist here!


1) The motivation currently explicitly states that the goal is to improve

performance for heavy state application. It seems that the motivation can

be stronger with the following use-case. Currently for MirrorMaker cluster

with e.g. 100 MirrorMaker processes, it will take a long time to rolling

bounce the entire MirrorMaker cluster. Each MirrorMaker process restart

will trigger a rebalance which currently pause the consumption of the all

partitions of the MirrorMaker cluster. With the change stated in this

patch, as long as a MirrorMaker can restart within the specified timeout

(e.g. 2 minutes), then we only need constant number of rebalance (e.g. for

leader restart) for the entire rolling bounce, which will significantly

improves the availability of the MirrorMaker pipeline. In my opinion, the

main benefit of the KIP is to avoid unnecessary rebalance if the consumer

process can be restarted within soon, which helps performance even if

overhead of state shuffling for a given process is small.

I just rephrased this part and added it to the KIP. Thanks for making the motivation more solid!

2) In order to simplify the KIP reading, can you follow the writeup style
of other KIP (e.g. KIP-98) and list the interface change such as new
configs (e.g. registration timeout), new request/response, new AdminClient
API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
these are specified in the Proposed Change section which makes it a bit
inconvenient to understand the new interface that will be exposed to user.
Explanation of the current two-phase rebalance protocol probably can be
moved out of public interface section.
This is a great suggestion! I just consolidated all the public API changes, and the whole KIP
looks much more organized!

3) There are currently two version of JoinGroupRequest in the KIP and only
one of them has field memberId. This seems confusing.
Yep, I already found this issue and fixed it.

4) It is mentioned in the KIP that "An admin API to force rebalance could
be helpful here, but we will make a call once we finished the major
implementation". So this seems to be still an open question in the current
design. We probably want to agree on this before voting for the KIP.
We have finalized the idea that this API is needed.

5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
specify the name of the config key and the default config value? Possible
default values include empty string or null (similar to transaction.id<https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Ftransaction.id%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3Db2d8sQWM8niJreqST7%252BJLcxfEyBmj7cJp4Lm5cYT57s%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=sA3XnqHD2s8ya93rOBg5IZvJmbkUqliBGQ3ouyOAUFk%3D&amp;reserved=0> in
producer config).
I have defined the `member.name` in "New configuration" section.

6) Regarding the use of the topic "static_member_map" to persist member
name map, currently if consumer coordinator broker goes offline, rebalance
is triggered and consumers will try connect to the new coordinator. If
these consumers can connect to the new coordinator within
max.poll.interval.ms<https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=INHjX2K9iIfEBC9eUvuxLWuiELaE1uSfr4BmxWyfusg%3D&amp;reserved=0> which by default is 5 minutes, given that broker can
use a deterministic algorithm to determine the partition -> member_name
mapping, each consumer should get assigned the same set of partitions
without requiring state shuffling. So it is not clear whether we have a
strong use-case for this new logic. Can you help clarify what is the
benefit of using topic "static_member_map" to persist member name map?
I have discussed with Guozhang offline, and I believe reusing the current `_consumer_offsets`
topic is a better and unified solution.

7) Regarding the introduction of the expensionTimeoutMs config, it is
mentioned that "we are using expansion timeout to replace rebalance
timeout, which is configured by max.poll.intervals from client side, and
using registration timeout to replace session timeout". Currently the
default max.poll.interval.ms<https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Feur04.safelinks.protection.outlook.com%2F%3Furl%3Dhttp%253A%252F%252Fmax.poll.interval.ms%26data%3D02%257C01%257C%257Cb48d52bf63324bd91a5208d64f43247d%257C84df9e7fe9f640afb435aaaaaaaaaaaa%257C1%257C0%257C636783547118328245%26sdata%3DJWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%253D%26reserved%3D0&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=INHjX2K9iIfEBC9eUvuxLWuiELaE1uSfr4BmxWyfusg%3D&amp;reserved=0> is configured to be 5 minutes and there will
be only one rebalance if all new consumers can join within 5 minutes. So it
is not clear whether we have a strong use-case for this new config. Can you
explain what is the benefit of introducing this new config?
Previously our goal is to use expansion timeout as a workaround for triggering multiple
rebalances when scaling up members are not joining at the same time. It is decided to
be addressed by client side protocol change, so we will not introduce expansion timeout.

8) It is mentioned that "To distinguish between previous version of
protocol, we will also increase the join group request version to v4 when
MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
the join group request shall be downgraded to v3 without setting the member
Id". It is probably simpler to just say that this feature is enabled if
JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
is configured with non-empty string.
Yep, addressed this!

9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
in OffsetCommitResponse for "commit requests under static membership". Can
you clarify how broker determines whether the commit request is under
static membership?

We have agreed that commit request shouldn't be affected by the new membership, thus
removing it here. Thanks for catching this!

Let me know if you have further suggestions or concerns. Thank you for your valuable feedback
to help me design the KIP better! (And I will try to address your feedbacks in next round Mayuresh 😊)

Best,
Boyang
________________________________
From: Mayuresh Gharat <gh...@gmail.com>
Sent: Wednesday, November 21, 2018 7:50 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

Thanks for updating the KIP. This is a step good direction for stateful
applications and also mirroring applications whose latency is affected due
to the rebalance issues that we have today.

I had a few questions on the current version of the KIP :
For the effectiveness of the KIP, consumer with member.name set will *not
send leave group request* when they go offline

> By this you mean, even if the application has not called
> KafkaConsumer.poll() within session timeout, it will not be sending the
> LeaveGroup request, right?
>

Broker will maintain an in-memory mapping of {member.name → member.id} to
track member uniqueness.

> When is the member.name removed from this map?
>

Member.id must be set if the *member.name <https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=C7pxl8FcLQamaFFJQ88OhUN7ATIa8GimdJDhSq6gxug%3D&amp;reserved=0> *is already
within the map. Otherwise reply MISSING_MEMBER_ID

> How is this case handled on the client side? What is the application that
> is using the KafkaConsumer suppose to do in this scenario?
>

Session timeout is the timeout we will trigger rebalance when a member goes
offline for too long (not sending heartbeat request). To make static
membership effective, we should increase the default max session timeout to
30 min so that end user could config it freely.

> This would mean that it might take more time to detect unowned topic
> partitions and may cause delay for applications that perform data mirroring
> tasks. I discussed this with our sre and we have a suggestion to make here
> as listed below separately.
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups

> you meant remove unjoined members of the group, right ?
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups. This is actually conflicting with the design of
static membership, because those temporarily unavailable members will
potentially reattempt the join group and trigger extra rebalances.
Internally we would optimize this logic by having rebalance timeout only in
charge of stopping prepare rebalance stage, without removing non-responsive
members immediately.

> What do you mean by " Internally we would optimize this logic by having
> rebalance timeout only in charge of stopping prepare rebalance stage,
> without removing non-responsive members immediately." There would not be a
> full rebalance if the lagging consumer sent a JoinGroup request later,
> right ? If yes, can you highlight this in the KIP ?
>

Scale Up

> The KIP talks about scale up scenario but its not quite clear how we
> handle it. Are we adding a separate "expansion.timeout" or we adding status
> "learner" ?. Can you shed more light on how this is handled in the KIP, if
> its handled?
>


*Discussion*
Larger session timeouts causing latency rise for getting data for un-owned
topic partitions :

> I think Jason had brought this up earlier about having a way to say how
> many members/consumer hosts are you choosing to be in the consumer group.
> If we can do this, then in case of mirroring applications we can do this :
> Lets say we have a mirroring application that consumes from Kafka cluster
> A and produces to Kafka cluster B.
> Depending on the data and the Kafka cluster configuration, Kafka service
> providers can set a mirroring group saying that it will take, for example
> 300 consumer hosts/members to achieve the desired throughput and latency
> for mirroring and can have additional 10 consumer hosts as spare in the
> same group.
> So when the first 300 members/consumers to join the group will start
> mirroring the data from Kafka cluster A to Kafka cluster B.
> The remaining 10 consumer members can sit idle.
> The moment one of the consumer (for example: consumer number 54) from the
> first 300 members go out of the group (crossed session timeout), it (the
> groupCoordinator) can just assign the topicPartitions from the consumer
> member 54 to one of the spare hosts.
> Once the consumer member 54 comes back up, it can start as being a part of
> the spare pool.
> This enables us to have lower session timeouts and low latency mirroring,
> in cases where the service providers are OK with having spare hosts.
> This would mean that we would tolerate n consumer members leaving and
> rejoining the group and still provide low latency as long as n <= number of
> spare consumers.
> If there are no spare host available, we can get back to the idea as
> described in the KIP.
>

Thanks,

Mayuresh





On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Hi Boyang.
>
> Thanks for preparing this KIP! It is making good progress and will be a
> great improvement for stateful Kafka applications.
>
> Apologies for my late reply, I was away for a while. Lots of great comments
> so far, so I'll probably second most of them in what I suggest below at
> this point.
>
> When I first read the KIP, I wanted to start at the end with something that
> wasn't highlighted a lot. That was the topic related to handling duplicate
> members. I see now that the initial suggestion of handling this situation
> during offset commit has been removed, and I agree with that. Issues
> related to membership seem to be handled better when the member joins the
> group rather than when it tries to commit offsets. This also simplifies how
> many request types need to change in order to incorporate the new member
> name field.
>
> I also agree with what Jason and Guozhang have said regarding timeouts.
> Although semantically, it's easier to think of every operation having its
> own timeout, operationally this can become a burden. Thus, consolidation
> seems preferable here. The definition of embedded protocols on top of the
> base group membership protocol for rebalancing gives enough flexibility to
> address such needs in each client component separately.
>
> Finally, some minor comments:
> In a few places the new/proposed changes are referred to as "current".
> Which is a bit confusing considering that there is a protocol in place
> already, and by "current" someone might understand the existing one. I'd
> recommend using new/proposed or equivalent when referring to changes
> introduced with KIP-345 and current/existing or equivalent when referring
> to existing behavior.
>
> There's the following sentence in the "Public Interfaces" section:
> "Since for many stateful consumer/stream applications, the state shuffling
> is more painful than short time partial unavailability."
> However, my understanding is that the changes proposed with KIP-345 will
> not exploit any partial availability. A suggestion for dealing with
> temporary imbalances has been made in "Incremental Cooperative Rebalancing"
> which can work well with KIP-345, but here I don't see proposed changes
> that suggest that some resources (e.g. partitions) will keep being used
> while others will not be utilized. Thus, you might want to adjust this
> sentence. Correct me if I'm missing something related to that.
>
> In the rejected alternatives, under point 2) I read "we can copy the member
> id to the config files". I believe it means to say "member name" unless I'm
> missing something about reusing member ids. Also below I read: "By allowing
> consumers to optionally specifying a member id" which probably implies
> "member name" again. In a sense this section highlights a potential
> confusion between member name and member id. I wonder if we could come up
> with a better term for the new field. StaticTag, StaticLabel, or even
> StaticName are some suggestions that could potentially help with confusion
> between MemberId and MemberName and what corresponds to what. But I
> wouldn't like to disrupt the discussion with naming conventions too much at
> this point. I just mention it here as a thought.
>
> Looking forward to see the final details of this KIP. Great work so far!
>
> Konstantine
>
>
> On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Guozhang for the great summary here, and I have been following up
> > the action items here.
> >
> >
> >   1.  I already updated the KIP to remove the expansion timeout and
> > registration timeout. Great to see them being addressed in client side!
> >   2.  I double checked the design and I believe that it is ok to have
> both
> > static member and dynamic member co-exist in the same group. So the
> upgrade
> > shouldn't be destructive and we are removing the two membership protocol
> > switching APIs.
> >   3.  I only have question about this one. I'm still reading the
> KafkaApis
> > code here. Should I just use the same authorization logic for
> > ForceStaticRebalanceRequest as JoinGroupRequest?
> >   4.  I'm very excited to see this work with K8! Like you suggested, this
> > feature could be better addressed in a separate KIP because it is pretty
> > independent. I could start drafting the KIP once the current proposal is
> > approved.
> >   5.  I believe that we don't need fencing in offset commit request,
> since
> > duplicate member.name issue could be handled by join group request. We
> > shall reject join group with known member name but no member id (which
> > means we already have an active member using this identity).
> >   6.  I agree to remove that internal config once we move forward with
> > static membership. And I already removed the entire section from the KIP.
> >
> > Let me know if you have other concerns.
> >
> > Best,
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Tuesday, November 20, 2018 4:21 PM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hello Boyang,
> >
> > Thanks a lot for the KIP! It is a great write-up and I appreciate your
> > patience answering to the feedbacks from the community. I'd like to add
> my
> > 2cents here:
> >
> > 1. By introducing another two timeout configs, registration_timeout and
> > expansion_timeout, we are effectively having four timeout configs:
> session
> > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> client
> > side), and these two. Interplaying these timeout configs can be quite
> hard
> > for users with such complexity, and hence I'm wondering if we can
> simplify
> > the situation with as less possible timeout configs as possible. Here is
> a
> > concrete suggestion I'd like propose:
> >
> > 1.a) Instead of introducing a registration_timeout in addition to the
> > session_timeout for static members, we can just reuse the session_timeout
> > and ask users to set it to a larger value when they are upgrading a
> dynamic
> > client to a static client by setting the "member.name" at the same time.
> > By
> > default, the broker-side min.session.timeout is 6 seconds and
> > max.session.timeout is 5 minutes, which seems reasonable to me (we can of
> > course modify this broker config to enlarge the valid interval if we want
> > in practice). And then we should also consider removing the condition for
> > marking a client as failed if the rebalance timeout has reached while the
> > JoinGroup was not received, so that the semantics of session_timeout and
> > rebalance_timeout are totally separated: the former is only used to
> > determine if a consumer member of the group should be marked as failed
> and
> > kicked out of the group, and the latter is only used to determine the
> > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> other
> > words if a member did not send the JoinGroup in time of the
> > rebalance_timeout, we still include it in the new generation of the group
> > and use its old subscription info to send to leader for assignment. Later
> > if the member came back with HeartBeat request, we can still follow the
> > normal path to bring it to the latest generation while checking that its
> > sent JoinGroup request contains the same subscription info as we used to
> > assign the partitions previously (which should be likely the case in
> > practice). In addition, we should let static members to not send the
> > LeaveGroup request when it is gracefully shutdown, so that a static
> member
> > can only be leaving the group if its session has timed out, OR it has
> been
> > indicated to not exist in the group any more (details below).
> >
> > 1.b) We have a parallel discussion about Incremental Cooperative
> > Rebalancing, in which we will encode the "when to rebalance" logic at the
> > application level, instead of at the protocol level. By doing this we can
> > also enable a few other optimizations, e.g. at the Streams level to first
> > build up the state store as standby tasks and then trigger a second
> > rebalance to actually migrate the active tasks while keeping the actual
> > rebalance latency and hence unavailability window to be small (
> >
> >
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=wrqKK%2BWMoA%2BF0fY%2BergbA73YTV6twD8lpcz1RYOvMKw%3D&amp;reserved=0
> ).
> > I'd propose we align
> > KIP-345 along with this idea, and hence do not add the expansion_timeout
> as
> > part of the protocol layer, but only do that at the application's
> > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > deprecate the "*group.initial.rebalance.delay.ms
> > <
> >
> https://nam04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C2f2dfc69b3694536124908d6504dbc3c%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636784692128528166&amp;sdata=q7SAR3tHJoFW0qZwwOdGk9FFNMIbUzPJfhR9iPuXPOM%3D&amp;reserved=0
> >*"
> > though as part of this KIP
> > since we have discussed about its limit and think it is actually not a
> very
> > good design and could be replaced with client-side logic above.
> >
> >
> > 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> > specifically, let's say after we have upgraded broker version to be able
> to
> > recognize the new versions of JoinGroup request and the admin requests,
> how
> > should we upgrade the clients and enable static groups? On top of my head
> > if we do a rolling bounce in which we set the member.name config as well
> > as
> > optionally increase the session.timeout config when we bounce each
> > instance, then during this rolling bounces we will have a group contained
> > with both dynamic members and static members. It means that we should
> have
> > the group to allow such scenario (i.e. we cannot reject JoinGroup
> requests
> > from dynamic members), and hence the "member.name" -> "member.id"
> mapping
> > will only be partial at this scenario. Also could you describe if the
> > upgrade to the first version that support this feature would ever get any
> > benefits, or only the future upgrade path for rolling bounces could get
> > benefits out of this feature?
> >
> > If that's the case and we will do 1) as suggested above, do we still need
> > the enableStaticMembership and enableDynamicMembership admin requests any
> > more? Seems it is not necessary any more as we will only have the notion
> of
> > "dynamic or static members" that can co-exist in a group while there no
> > notion of "dynamic or static groups", and hence these two requests are
> not
> > needed anymore.
> >
> >
> > 3. We need to briefly talk about the implications for ACL as we introduce
> > new admin requests that are related to a specific group.id. For example,
> > we
> > need to make sure that whoever created the group or joined the group can
> > actually send admin requests for the group, otherwise the application
> > owners need to bother the Kafka operators on a multi-tenant cluster every
> > time they want to send any admin requests for their groups which would be
> > an operational nightmare.
> >
> >
> > 4. I like Jason's suggestion of adding an optional field for the list of
> > member names, and I'm wondering if that can be done as part of the
> > forceStaticRebalance request: i.e. by passing a list of members, we will
> > enforce a rebalance immediately since it indicates that some static
> member
> > will be officially kicked out of the group and some new static members
> may
> > be added. So back to 1.a) above, a static member can only be kicked out
> of
> > the group if a) its session (arguably long period of time) has timed out,
> > and b) this admin request explicitly state that it is no longer part of
> the
> > group. As for execution I'm fine with keeping it as a future work of this
> > KIP if you'd like to make its scope smaller.
> >
> > Following are minor comments:
> >
> > 5. I'm not sure if we need to include "member.name" as part of the
> > OffsetCommitRequest for fencing purposes, as I think the memberId plus
> the
> > generation number should be sufficient for fencing even with static
> > members.
> >
> > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Boyang,
> > >
> > > Thanks for the proposal! This is very useful. I have some comments
> below:
> > >
> > > 1) The motivation currently explicitly states that the goal is to
> improve
> > > performance for heavy state application. It seems that the motivation
> can
> > > be stronger with the following use-case. Currently for MirrorMaker
> > cluster
> > > with e.g. 100 MirrorMaker processes, it will take a long time to
> rolling
> > > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> > > will trigger a rebalance which currently pause the consumption of the
> all
> > > partitions of the MirrorMaker cluster. With the change stated in this
> > > patch, as long as a MirrorMaker can restart within the specified
> timeout
> > > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> > for
> > > leader restart) for the entire rolling bounce, which will significantly
> > > improves the availability of the MirrorMaker pipeline. In my opinion,
> the
> > > main benefit of the KIP is to avoid unnecessary rebalance if the
> consumer
> > > process can be restarted within soon, which helps performance even if
> > > overhead of state shuffling for a given process is small.
> > >
> > > 2) In order to simplify the KIP reading, can you follow the writeup
> style
> > > of other KIP (e.g. KIP-98) and list the interface change such as new
> > > configs (e.g. registration timeout), new request/response, new
> > AdminClient
> > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> of
> > > these are specified in the Proposed Change section which makes it a bit
> > > inconvenient to understand the new interface that will be exposed to
> > user.
> > > Explanation of the current two-phase rebalance protocol probably can be
> > > moved out of public interface section.
> > >
> > > 3) There are currently two version of JoinGroupRequest in the KIP and
> > only
> > > one of them has field memberId. This seems confusing.
> > >
> > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> could
> > > be helpful here, but we will make a call once we finished the major
> > > implementation". So this seems to be still an open question in the
> > current
> > > design. We probably want to agree on this before voting for the KIP.
> > >
> > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > > specify the name of the config key and the default config value?
> Possible
> > > default values include empty string or null (similar to transaction.id
> > in
> > > producer config).
> > >
> > > 6) Regarding the use of the topic "static_member_map" to persist member
> > > name map, currently if consumer coordinator broker goes offline,
> > rebalance
> > > is triggered and consumers will try connect to the new coordinator. If
> > > these consumers can connect to the new coordinator within
> > > max.poll.interval.ms which by default is 5 minutes, given that broker
> > can
> > > use a deterministic algorithm to determine the partition -> member_name
> > > mapping, each consumer should get assigned the same set of partitions
> > > without requiring state shuffling. So it is not clear whether we have a
> > > strong use-case for this new logic. Can you help clarify what is the
> > > benefit of using topic "static_member_map" to persist member name map?
> > >
> > > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > > mentioned that "we are using expansion timeout to replace rebalance
> > > timeout, which is configured by max.poll.intervals from client side,
> and
> > > using registration timeout to replace session timeout". Currently the
> > > default max.poll.interval.ms is configured to be 5 minutes and there
> > will
> > > be only one rebalance if all new consumers can join within 5 minutes.
> So
> > it
> > > is not clear whether we have a strong use-case for this new config. Can
> > you
> > > explain what is the benefit of introducing this new config?
> > >
> > > 8) It is mentioned that "To distinguish between previous version of
> > > protocol, we will also increase the join group request version to v4
> when
> > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> v4),
> > > the join group request shall be downgraded to v3 without setting the
> > member
> > > Id". It is probably simpler to just say that this feature is enabled if
> > > JoinGroupRequest V4 is supported on both client and broker and
> > MEMBER_NAME
> > > is configured with non-empty string.
> > >
> > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> error
> > > in OffsetCommitResponse for "commit requests under static membership".
> > Can
> > > you clarify how broker determines whether the commit request is under
> > > static membership?
> > >
> > > Thanks,
> > > Dong
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


--
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Hey Dong, sorry for missing your message. I couldn't find your email on my thread, so I will just do a checklist here!


1) The motivation currently explicitly states that the goal is to improve

performance for heavy state application. It seems that the motivation can

be stronger with the following use-case. Currently for MirrorMaker cluster

with e.g. 100 MirrorMaker processes, it will take a long time to rolling

bounce the entire MirrorMaker cluster. Each MirrorMaker process restart

will trigger a rebalance which currently pause the consumption of the all

partitions of the MirrorMaker cluster. With the change stated in this

patch, as long as a MirrorMaker can restart within the specified timeout

(e.g. 2 minutes), then we only need constant number of rebalance (e.g. for

leader restart) for the entire rolling bounce, which will significantly

improves the availability of the MirrorMaker pipeline. In my opinion, the

main benefit of the KIP is to avoid unnecessary rebalance if the consumer

process can be restarted within soon, which helps performance even if

overhead of state shuffling for a given process is small.

I just rephrased this part and added it to the KIP. Thanks for making the motivation more solid!

2) In order to simplify the KIP reading, can you follow the writeup style
of other KIP (e.g. KIP-98) and list the interface change such as new
configs (e.g. registration timeout), new request/response, new AdminClient
API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
these are specified in the Proposed Change section which makes it a bit
inconvenient to understand the new interface that will be exposed to user.
Explanation of the current two-phase rebalance protocol probably can be
moved out of public interface section.
This is a great suggestion! I just consolidated all the public API changes, and the whole KIP
looks much more organized!

3) There are currently two version of JoinGroupRequest in the KIP and only
one of them has field memberId. This seems confusing.
Yep, I already found this issue and fixed it.

4) It is mentioned in the KIP that "An admin API to force rebalance could
be helpful here, but we will make a call once we finished the major
implementation". So this seems to be still an open question in the current
design. We probably want to agree on this before voting for the KIP.
We have finalized the idea that this API is needed.

5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
specify the name of the config key and the default config value? Possible
default values include empty string or null (similar to transaction.id<https://eur04.safelinks.protection.outlook.com/?url=http%3A%2F%2Ftransaction.id&data=02%7C01%7C%7Cb48d52bf63324bd91a5208d64f43247d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783547118328245&sdata=b2d8sQWM8niJreqST7%2BJLcxfEyBmj7cJp4Lm5cYT57s%3D&reserved=0> in
producer config).
I have defined the `member.name` in "New configuration" section.

6) Regarding the use of the topic "static_member_map" to persist member
name map, currently if consumer coordinator broker goes offline, rebalance
is triggered and consumers will try connect to the new coordinator. If
these consumers can connect to the new coordinator within
max.poll.interval.ms<https://eur04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmax.poll.interval.ms&data=02%7C01%7C%7Cb48d52bf63324bd91a5208d64f43247d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783547118328245&sdata=JWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%3D&reserved=0> which by default is 5 minutes, given that broker can
use a deterministic algorithm to determine the partition -> member_name
mapping, each consumer should get assigned the same set of partitions
without requiring state shuffling. So it is not clear whether we have a
strong use-case for this new logic. Can you help clarify what is the
benefit of using topic "static_member_map" to persist member name map?
I have discussed with Guozhang offline, and I believe reusing the current `_consumer_offsets`
topic is a better and unified solution.

7) Regarding the introduction of the expensionTimeoutMs config, it is
mentioned that "we are using expansion timeout to replace rebalance
timeout, which is configured by max.poll.intervals from client side, and
using registration timeout to replace session timeout". Currently the
default max.poll.interval.ms<https://eur04.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmax.poll.interval.ms&data=02%7C01%7C%7Cb48d52bf63324bd91a5208d64f43247d%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783547118328245&sdata=JWiSn5gQO5VNrmBov0KBdHpyVb4CiA0pFOAtLAlFqqY%3D&reserved=0> is configured to be 5 minutes and there will
be only one rebalance if all new consumers can join within 5 minutes. So it
is not clear whether we have a strong use-case for this new config. Can you
explain what is the benefit of introducing this new config?
Previously our goal is to use expansion timeout as a workaround for triggering multiple
rebalances when scaling up members are not joining at the same time. It is decided to
be addressed by client side protocol change, so we will not introduce expansion timeout.

8) It is mentioned that "To distinguish between previous version of
protocol, we will also increase the join group request version to v4 when
MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
the join group request shall be downgraded to v3 without setting the member
Id". It is probably simpler to just say that this feature is enabled if
JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
is configured with non-empty string.
Yep, addressed this!

9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
in OffsetCommitResponse for "commit requests under static membership". Can
you clarify how broker determines whether the commit request is under
static membership?

We have agreed that commit request shouldn't be affected by the new membership, thus
removing it here. Thanks for catching this!

Let me know if you have further suggestions or concerns. Thank you for your valuable feedback
to help me design the KIP better! (And I will try to address your feedbacks in next round Mayuresh 😊)

Best,
Boyang
________________________________
From: Mayuresh Gharat <gh...@gmail.com>
Sent: Wednesday, November 21, 2018 7:50 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

Thanks for updating the KIP. This is a step good direction for stateful
applications and also mirroring applications whose latency is affected due
to the rebalance issues that we have today.

I had a few questions on the current version of the KIP :
For the effectiveness of the KIP, consumer with member.name set will *not
send leave group request* when they go offline

> By this you mean, even if the application has not called
> KafkaConsumer.poll() within session timeout, it will not be sending the
> LeaveGroup request, right?
>

Broker will maintain an in-memory mapping of {member.name → member.id} to
track member uniqueness.

> When is the member.name removed from this map?
>

Member.id must be set if the *member.name <https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fmember.name&amp;data=02%7C01%7C%7Cd9b758a0a8be4f85e16c08d64f430e40%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783546751462147&amp;sdata=39mD7qTQCN6ZB04ilVcHdlngjicF5335YBun9n9HIR8%3D&amp;reserved=0> *is already
within the map. Otherwise reply MISSING_MEMBER_ID

> How is this case handled on the client side? What is the application that
> is using the KafkaConsumer suppose to do in this scenario?
>

Session timeout is the timeout we will trigger rebalance when a member goes
offline for too long (not sending heartbeat request). To make static
membership effective, we should increase the default max session timeout to
30 min so that end user could config it freely.

> This would mean that it might take more time to detect unowned topic
> partitions and may cause delay for applications that perform data mirroring
> tasks. I discussed this with our sre and we have a suggestion to make here
> as listed below separately.
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups

> you meant remove unjoined members of the group, right ?
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups. This is actually conflicting with the design of
static membership, because those temporarily unavailable members will
potentially reattempt the join group and trigger extra rebalances.
Internally we would optimize this logic by having rebalance timeout only in
charge of stopping prepare rebalance stage, without removing non-responsive
members immediately.

> What do you mean by " Internally we would optimize this logic by having
> rebalance timeout only in charge of stopping prepare rebalance stage,
> without removing non-responsive members immediately." There would not be a
> full rebalance if the lagging consumer sent a JoinGroup request later,
> right ? If yes, can you highlight this in the KIP ?
>

Scale Up

> The KIP talks about scale up scenario but its not quite clear how we
> handle it. Are we adding a separate "expansion.timeout" or we adding status
> "learner" ?. Can you shed more light on how this is handled in the KIP, if
> its handled?
>


*Discussion*
Larger session timeouts causing latency rise for getting data for un-owned
topic partitions :

> I think Jason had brought this up earlier about having a way to say how
> many members/consumer hosts are you choosing to be in the consumer group.
> If we can do this, then in case of mirroring applications we can do this :
> Lets say we have a mirroring application that consumes from Kafka cluster
> A and produces to Kafka cluster B.
> Depending on the data and the Kafka cluster configuration, Kafka service
> providers can set a mirroring group saying that it will take, for example
> 300 consumer hosts/members to achieve the desired throughput and latency
> for mirroring and can have additional 10 consumer hosts as spare in the
> same group.
> So when the first 300 members/consumers to join the group will start
> mirroring the data from Kafka cluster A to Kafka cluster B.
> The remaining 10 consumer members can sit idle.
> The moment one of the consumer (for example: consumer number 54) from the
> first 300 members go out of the group (crossed session timeout), it (the
> groupCoordinator) can just assign the topicPartitions from the consumer
> member 54 to one of the spare hosts.
> Once the consumer member 54 comes back up, it can start as being a part of
> the spare pool.
> This enables us to have lower session timeouts and low latency mirroring,
> in cases where the service providers are OK with having spare hosts.
> This would mean that we would tolerate n consumer members leaving and
> rejoining the group and still provide low latency as long as n <= number of
> spare consumers.
> If there are no spare host available, we can get back to the idea as
> described in the KIP.
>

Thanks,

Mayuresh





On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Hi Boyang.
>
> Thanks for preparing this KIP! It is making good progress and will be a
> great improvement for stateful Kafka applications.
>
> Apologies for my late reply, I was away for a while. Lots of great comments
> so far, so I'll probably second most of them in what I suggest below at
> this point.
>
> When I first read the KIP, I wanted to start at the end with something that
> wasn't highlighted a lot. That was the topic related to handling duplicate
> members. I see now that the initial suggestion of handling this situation
> during offset commit has been removed, and I agree with that. Issues
> related to membership seem to be handled better when the member joins the
> group rather than when it tries to commit offsets. This also simplifies how
> many request types need to change in order to incorporate the new member
> name field.
>
> I also agree with what Jason and Guozhang have said regarding timeouts.
> Although semantically, it's easier to think of every operation having its
> own timeout, operationally this can become a burden. Thus, consolidation
> seems preferable here. The definition of embedded protocols on top of the
> base group membership protocol for rebalancing gives enough flexibility to
> address such needs in each client component separately.
>
> Finally, some minor comments:
> In a few places the new/proposed changes are referred to as "current".
> Which is a bit confusing considering that there is a protocol in place
> already, and by "current" someone might understand the existing one. I'd
> recommend using new/proposed or equivalent when referring to changes
> introduced with KIP-345 and current/existing or equivalent when referring
> to existing behavior.
>
> There's the following sentence in the "Public Interfaces" section:
> "Since for many stateful consumer/stream applications, the state shuffling
> is more painful than short time partial unavailability."
> However, my understanding is that the changes proposed with KIP-345 will
> not exploit any partial availability. A suggestion for dealing with
> temporary imbalances has been made in "Incremental Cooperative Rebalancing"
> which can work well with KIP-345, but here I don't see proposed changes
> that suggest that some resources (e.g. partitions) will keep being used
> while others will not be utilized. Thus, you might want to adjust this
> sentence. Correct me if I'm missing something related to that.
>
> In the rejected alternatives, under point 2) I read "we can copy the member
> id to the config files". I believe it means to say "member name" unless I'm
> missing something about reusing member ids. Also below I read: "By allowing
> consumers to optionally specifying a member id" which probably implies
> "member name" again. In a sense this section highlights a potential
> confusion between member name and member id. I wonder if we could come up
> with a better term for the new field. StaticTag, StaticLabel, or even
> StaticName are some suggestions that could potentially help with confusion
> between MemberId and MemberName and what corresponds to what. But I
> wouldn't like to disrupt the discussion with naming conventions too much at
> this point. I just mention it here as a thought.
>
> Looking forward to see the final details of this KIP. Great work so far!
>
> Konstantine
>
>
> On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Guozhang for the great summary here, and I have been following up
> > the action items here.
> >
> >
> >   1.  I already updated the KIP to remove the expansion timeout and
> > registration timeout. Great to see them being addressed in client side!
> >   2.  I double checked the design and I believe that it is ok to have
> both
> > static member and dynamic member co-exist in the same group. So the
> upgrade
> > shouldn't be destructive and we are removing the two membership protocol
> > switching APIs.
> >   3.  I only have question about this one. I'm still reading the
> KafkaApis
> > code here. Should I just use the same authorization logic for
> > ForceStaticRebalanceRequest as JoinGroupRequest?
> >   4.  I'm very excited to see this work with K8! Like you suggested, this
> > feature could be better addressed in a separate KIP because it is pretty
> > independent. I could start drafting the KIP once the current proposal is
> > approved.
> >   5.  I believe that we don't need fencing in offset commit request,
> since
> > duplicate member.name issue could be handled by join group request. We
> > shall reject join group with known member name but no member id (which
> > means we already have an active member using this identity).
> >   6.  I agree to remove that internal config once we move forward with
> > static membership. And I already removed the entire section from the KIP.
> >
> > Let me know if you have other concerns.
> >
> > Best,
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Tuesday, November 20, 2018 4:21 PM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hello Boyang,
> >
> > Thanks a lot for the KIP! It is a great write-up and I appreciate your
> > patience answering to the feedbacks from the community. I'd like to add
> my
> > 2cents here:
> >
> > 1. By introducing another two timeout configs, registration_timeout and
> > expansion_timeout, we are effectively having four timeout configs:
> session
> > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> client
> > side), and these two. Interplaying these timeout configs can be quite
> hard
> > for users with such complexity, and hence I'm wondering if we can
> simplify
> > the situation with as less possible timeout configs as possible. Here is
> a
> > concrete suggestion I'd like propose:
> >
> > 1.a) Instead of introducing a registration_timeout in addition to the
> > session_timeout for static members, we can just reuse the session_timeout
> > and ask users to set it to a larger value when they are upgrading a
> dynamic
> > client to a static client by setting the "member.name" at the same time.
> > By
> > default, the broker-side min.session.timeout is 6 seconds and
> > max.session.timeout is 5 minutes, which seems reasonable to me (we can of
> > course modify this broker config to enlarge the valid interval if we want
> > in practice). And then we should also consider removing the condition for
> > marking a client as failed if the rebalance timeout has reached while the
> > JoinGroup was not received, so that the semantics of session_timeout and
> > rebalance_timeout are totally separated: the former is only used to
> > determine if a consumer member of the group should be marked as failed
> and
> > kicked out of the group, and the latter is only used to determine the
> > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> other
> > words if a member did not send the JoinGroup in time of the
> > rebalance_timeout, we still include it in the new generation of the group
> > and use its old subscription info to send to leader for assignment. Later
> > if the member came back with HeartBeat request, we can still follow the
> > normal path to bring it to the latest generation while checking that its
> > sent JoinGroup request contains the same subscription info as we used to
> > assign the partitions previously (which should be likely the case in
> > practice). In addition, we should let static members to not send the
> > LeaveGroup request when it is gracefully shutdown, so that a static
> member
> > can only be leaving the group if its session has timed out, OR it has
> been
> > indicated to not exist in the group any more (details below).
> >
> > 1.b) We have a parallel discussion about Incremental Cooperative
> > Rebalancing, in which we will encode the "when to rebalance" logic at the
> > application level, instead of at the protocol level. By doing this we can
> > also enable a few other optimizations, e.g. at the Streams level to first
> > build up the state store as standby tasks and then trigger a second
> > rebalance to actually migrate the active tasks while keeping the actual
> > rebalance latency and hence unavailability window to be small (
> >
> >
> https://eur03.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7Cd9b758a0a8be4f85e16c08d64f430e40%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783546751462147&amp;sdata=vIhqG33RYHgtP%2FtM9zstA0qxzVxsBed%2Biy43zfPb4vg%3D&amp;reserved=0
> ).
> > I'd propose we align
> > KIP-345 along with this idea, and hence do not add the expansion_timeout
> as
> > part of the protocol layer, but only do that at the application's
> > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > deprecate the "*group.initial.rebalance.delay.ms
> > <
> >
> https://eur03.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7Cd9b758a0a8be4f85e16c08d64f430e40%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636783546751462147&amp;sdata=9%2F0yWjIjFVN9q5x71H2Qy%2FVCHAVdFo2%2BRtTM12jYBJs%3D&amp;reserved=0
> >*"
> > though as part of this KIP
> > since we have discussed about its limit and think it is actually not a
> very
> > good design and could be replaced with client-side logic above.
> >
> >
> > 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> > specifically, let's say after we have upgraded broker version to be able
> to
> > recognize the new versions of JoinGroup request and the admin requests,
> how
> > should we upgrade the clients and enable static groups? On top of my head
> > if we do a rolling bounce in which we set the member.name config as well
> > as
> > optionally increase the session.timeout config when we bounce each
> > instance, then during this rolling bounces we will have a group contained
> > with both dynamic members and static members. It means that we should
> have
> > the group to allow such scenario (i.e. we cannot reject JoinGroup
> requests
> > from dynamic members), and hence the "member.name" -> "member.id"
> mapping
> > will only be partial at this scenario. Also could you describe if the
> > upgrade to the first version that support this feature would ever get any
> > benefits, or only the future upgrade path for rolling bounces could get
> > benefits out of this feature?
> >
> > If that's the case and we will do 1) as suggested above, do we still need
> > the enableStaticMembership and enableDynamicMembership admin requests any
> > more? Seems it is not necessary any more as we will only have the notion
> of
> > "dynamic or static members" that can co-exist in a group while there no
> > notion of "dynamic or static groups", and hence these two requests are
> not
> > needed anymore.
> >
> >
> > 3. We need to briefly talk about the implications for ACL as we introduce
> > new admin requests that are related to a specific group.id. For example,
> > we
> > need to make sure that whoever created the group or joined the group can
> > actually send admin requests for the group, otherwise the application
> > owners need to bother the Kafka operators on a multi-tenant cluster every
> > time they want to send any admin requests for their groups which would be
> > an operational nightmare.
> >
> >
> > 4. I like Jason's suggestion of adding an optional field for the list of
> > member names, and I'm wondering if that can be done as part of the
> > forceStaticRebalance request: i.e. by passing a list of members, we will
> > enforce a rebalance immediately since it indicates that some static
> member
> > will be officially kicked out of the group and some new static members
> may
> > be added. So back to 1.a) above, a static member can only be kicked out
> of
> > the group if a) its session (arguably long period of time) has timed out,
> > and b) this admin request explicitly state that it is no longer part of
> the
> > group. As for execution I'm fine with keeping it as a future work of this
> > KIP if you'd like to make its scope smaller.
> >
> > Following are minor comments:
> >
> > 5. I'm not sure if we need to include "member.name" as part of the
> > OffsetCommitRequest for fencing purposes, as I think the memberId plus
> the
> > generation number should be sufficient for fencing even with static
> > members.
> >
> > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Boyang,
> > >
> > > Thanks for the proposal! This is very useful. I have some comments
> below:
> > >
> > > 1) The motivation currently explicitly states that the goal is to
> improve
> > > performance for heavy state application. It seems that the motivation
> can
> > > be stronger with the following use-case. Currently for MirrorMaker
> > cluster
> > > with e.g. 100 MirrorMaker processes, it will take a long time to
> rolling
> > > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> > > will trigger a rebalance which currently pause the consumption of the
> all
> > > partitions of the MirrorMaker cluster. With the change stated in this
> > > patch, as long as a MirrorMaker can restart within the specified
> timeout
> > > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> > for
> > > leader restart) for the entire rolling bounce, which will significantly
> > > improves the availability of the MirrorMaker pipeline. In my opinion,
> the
> > > main benefit of the KIP is to avoid unnecessary rebalance if the
> consumer
> > > process can be restarted within soon, which helps performance even if
> > > overhead of state shuffling for a given process is small.
> > >
> > > 2) In order to simplify the KIP reading, can you follow the writeup
> style
> > > of other KIP (e.g. KIP-98) and list the interface change such as new
> > > configs (e.g. registration timeout), new request/response, new
> > AdminClient
> > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> of
> > > these are specified in the Proposed Change section which makes it a bit
> > > inconvenient to understand the new interface that will be exposed to
> > user.
> > > Explanation of the current two-phase rebalance protocol probably can be
> > > moved out of public interface section.
> > >
> > > 3) There are currently two version of JoinGroupRequest in the KIP and
> > only
> > > one of them has field memberId. This seems confusing.
> > >
> > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> could
> > > be helpful here, but we will make a call once we finished the major
> > > implementation". So this seems to be still an open question in the
> > current
> > > design. We probably want to agree on this before voting for the KIP.
> > >
> > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > > specify the name of the config key and the default config value?
> Possible
> > > default values include empty string or null (similar to transaction.id
> > in
> > > producer config).
> > >
> > > 6) Regarding the use of the topic "static_member_map" to persist member
> > > name map, currently if consumer coordinator broker goes offline,
> > rebalance
> > > is triggered and consumers will try connect to the new coordinator. If
> > > these consumers can connect to the new coordinator within
> > > max.poll.interval.ms which by default is 5 minutes, given that broker
> > can
> > > use a deterministic algorithm to determine the partition -> member_name
> > > mapping, each consumer should get assigned the same set of partitions
> > > without requiring state shuffling. So it is not clear whether we have a
> > > strong use-case for this new logic. Can you help clarify what is the
> > > benefit of using topic "static_member_map" to persist member name map?
> > >
> > > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > > mentioned that "we are using expansion timeout to replace rebalance
> > > timeout, which is configured by max.poll.intervals from client side,
> and
> > > using registration timeout to replace session timeout". Currently the
> > > default max.poll.interval.ms is configured to be 5 minutes and there
> > will
> > > be only one rebalance if all new consumers can join within 5 minutes.
> So
> > it
> > > is not clear whether we have a strong use-case for this new config. Can
> > you
> > > explain what is the benefit of introducing this new config?
> > >
> > > 8) It is mentioned that "To distinguish between previous version of
> > > protocol, we will also increase the join group request version to v4
> when
> > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> v4),
> > > the join group request shall be downgraded to v3 without setting the
> > member
> > > Id". It is probably simpler to just say that this feature is enabled if
> > > JoinGroupRequest V4 is supported on both client and broker and
> > MEMBER_NAME
> > > is configured with non-empty string.
> > >
> > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> error
> > > in OffsetCommitResponse for "commit requests under static membership".
> > Can
> > > you clarify how broker determines whether the commit request is under
> > > static membership?
> > >
> > > Thanks,
> > > Dong
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


--
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Thanks for updating the KIP. This is a step good direction for stateful
applications and also mirroring applications whose latency is affected due
to the rebalance issues that we have today.

I had a few questions on the current version of the KIP :
For the effectiveness of the KIP, consumer with member.name set will *not
send leave group request* when they go offline

> By this you mean, even if the application has not called
> KafkaConsumer.poll() within session timeout, it will not be sending the
> LeaveGroup request, right?
>

Broker will maintain an in-memory mapping of {member.name → member.id} to
track member uniqueness.

> When is the member.name removed from this map?
>

Member.id must be set if the *member.name <http://member.name> *is already
within the map. Otherwise reply MISSING_MEMBER_ID

> How is this case handled on the client side? What is the application that
> is using the KafkaConsumer suppose to do in this scenario?
>

Session timeout is the timeout we will trigger rebalance when a member goes
offline for too long (not sending heartbeat request). To make static
membership effective, we should increase the default max session timeout to
30 min so that end user could config it freely.

> This would mean that it might take more time to detect unowned topic
> partitions and may cause delay for applications that perform data mirroring
> tasks. I discussed this with our sre and we have a suggestion to make here
> as listed below separately.
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups

> you meant remove unjoined members of the group, right ?
>

Currently there is a config called *rebalance timeout* which is configured
by consumer *max.poll.intervals*. The reason we set it to poll interval is
because consumer could only send request within the call of poll() and we
want to wait sufficient time for the join group request. When reaching
rebalance timeout, the group will move towards completingRebalance stage
and remove unjoined groups. This is actually conflicting with the design of
static membership, because those temporarily unavailable members will
potentially reattempt the join group and trigger extra rebalances.
Internally we would optimize this logic by having rebalance timeout only in
charge of stopping prepare rebalance stage, without removing non-responsive
members immediately.

> What do you mean by " Internally we would optimize this logic by having
> rebalance timeout only in charge of stopping prepare rebalance stage,
> without removing non-responsive members immediately." There would not be a
> full rebalance if the lagging consumer sent a JoinGroup request later,
> right ? If yes, can you highlight this in the KIP ?
>

Scale Up

> The KIP talks about scale up scenario but its not quite clear how we
> handle it. Are we adding a separate "expansion.timeout" or we adding status
> "learner" ?. Can you shed more light on how this is handled in the KIP, if
> its handled?
>


*Discussion*
Larger session timeouts causing latency rise for getting data for un-owned
topic partitions :

> I think Jason had brought this up earlier about having a way to say how
> many members/consumer hosts are you choosing to be in the consumer group.
> If we can do this, then in case of mirroring applications we can do this :
> Lets say we have a mirroring application that consumes from Kafka cluster
> A and produces to Kafka cluster B.
> Depending on the data and the Kafka cluster configuration, Kafka service
> providers can set a mirroring group saying that it will take, for example
> 300 consumer hosts/members to achieve the desired throughput and latency
> for mirroring and can have additional 10 consumer hosts as spare in the
> same group.
> So when the first 300 members/consumers to join the group will start
> mirroring the data from Kafka cluster A to Kafka cluster B.
> The remaining 10 consumer members can sit idle.
> The moment one of the consumer (for example: consumer number 54) from the
> first 300 members go out of the group (crossed session timeout), it (the
> groupCoordinator) can just assign the topicPartitions from the consumer
> member 54 to one of the spare hosts.
> Once the consumer member 54 comes back up, it can start as being a part of
> the spare pool.
> This enables us to have lower session timeouts and low latency mirroring,
> in cases where the service providers are OK with having spare hosts.
> This would mean that we would tolerate n consumer members leaving and
> rejoining the group and still provide low latency as long as n <= number of
> spare consumers.
> If there are no spare host available, we can get back to the idea as
> described in the KIP.
>

Thanks,

Mayuresh





On Tue, Nov 20, 2018 at 10:18 AM Konstantine Karantasis <
konstantine@confluent.io> wrote:

> Hi Boyang.
>
> Thanks for preparing this KIP! It is making good progress and will be a
> great improvement for stateful Kafka applications.
>
> Apologies for my late reply, I was away for a while. Lots of great comments
> so far, so I'll probably second most of them in what I suggest below at
> this point.
>
> When I first read the KIP, I wanted to start at the end with something that
> wasn't highlighted a lot. That was the topic related to handling duplicate
> members. I see now that the initial suggestion of handling this situation
> during offset commit has been removed, and I agree with that. Issues
> related to membership seem to be handled better when the member joins the
> group rather than when it tries to commit offsets. This also simplifies how
> many request types need to change in order to incorporate the new member
> name field.
>
> I also agree with what Jason and Guozhang have said regarding timeouts.
> Although semantically, it's easier to think of every operation having its
> own timeout, operationally this can become a burden. Thus, consolidation
> seems preferable here. The definition of embedded protocols on top of the
> base group membership protocol for rebalancing gives enough flexibility to
> address such needs in each client component separately.
>
> Finally, some minor comments:
> In a few places the new/proposed changes are referred to as "current".
> Which is a bit confusing considering that there is a protocol in place
> already, and by "current" someone might understand the existing one. I'd
> recommend using new/proposed or equivalent when referring to changes
> introduced with KIP-345 and current/existing or equivalent when referring
> to existing behavior.
>
> There's the following sentence in the "Public Interfaces" section:
> "Since for many stateful consumer/stream applications, the state shuffling
> is more painful than short time partial unavailability."
> However, my understanding is that the changes proposed with KIP-345 will
> not exploit any partial availability. A suggestion for dealing with
> temporary imbalances has been made in "Incremental Cooperative Rebalancing"
> which can work well with KIP-345, but here I don't see proposed changes
> that suggest that some resources (e.g. partitions) will keep being used
> while others will not be utilized. Thus, you might want to adjust this
> sentence. Correct me if I'm missing something related to that.
>
> In the rejected alternatives, under point 2) I read "we can copy the member
> id to the config files". I believe it means to say "member name" unless I'm
> missing something about reusing member ids. Also below I read: "By allowing
> consumers to optionally specifying a member id" which probably implies
> "member name" again. In a sense this section highlights a potential
> confusion between member name and member id. I wonder if we could come up
> with a better term for the new field. StaticTag, StaticLabel, or even
> StaticName are some suggestions that could potentially help with confusion
> between MemberId and MemberName and what corresponds to what. But I
> wouldn't like to disrupt the discussion with naming conventions too much at
> this point. I just mention it here as a thought.
>
> Looking forward to see the final details of this KIP. Great work so far!
>
> Konstantine
>
>
> On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Guozhang for the great summary here, and I have been following up
> > the action items here.
> >
> >
> >   1.  I already updated the KIP to remove the expansion timeout and
> > registration timeout. Great to see them being addressed in client side!
> >   2.  I double checked the design and I believe that it is ok to have
> both
> > static member and dynamic member co-exist in the same group. So the
> upgrade
> > shouldn't be destructive and we are removing the two membership protocol
> > switching APIs.
> >   3.  I only have question about this one. I'm still reading the
> KafkaApis
> > code here. Should I just use the same authorization logic for
> > ForceStaticRebalanceRequest as JoinGroupRequest?
> >   4.  I'm very excited to see this work with K8! Like you suggested, this
> > feature could be better addressed in a separate KIP because it is pretty
> > independent. I could start drafting the KIP once the current proposal is
> > approved.
> >   5.  I believe that we don't need fencing in offset commit request,
> since
> > duplicate member.name issue could be handled by join group request. We
> > shall reject join group with known member name but no member id (which
> > means we already have an active member using this identity).
> >   6.  I agree to remove that internal config once we move forward with
> > static membership. And I already removed the entire section from the KIP.
> >
> > Let me know if you have other concerns.
> >
> > Best,
> > Boyang
> > ________________________________
> > From: Guozhang Wang <wa...@gmail.com>
> > Sent: Tuesday, November 20, 2018 4:21 PM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hello Boyang,
> >
> > Thanks a lot for the KIP! It is a great write-up and I appreciate your
> > patience answering to the feedbacks from the community. I'd like to add
> my
> > 2cents here:
> >
> > 1. By introducing another two timeout configs, registration_timeout and
> > expansion_timeout, we are effectively having four timeout configs:
> session
> > timeout, rebalance timeout (configured as "max.poll.interval.ms" on
> client
> > side), and these two. Interplaying these timeout configs can be quite
> hard
> > for users with such complexity, and hence I'm wondering if we can
> simplify
> > the situation with as less possible timeout configs as possible. Here is
> a
> > concrete suggestion I'd like propose:
> >
> > 1.a) Instead of introducing a registration_timeout in addition to the
> > session_timeout for static members, we can just reuse the session_timeout
> > and ask users to set it to a larger value when they are upgrading a
> dynamic
> > client to a static client by setting the "member.name" at the same time.
> > By
> > default, the broker-side min.session.timeout is 6 seconds and
> > max.session.timeout is 5 minutes, which seems reasonable to me (we can of
> > course modify this broker config to enlarge the valid interval if we want
> > in practice). And then we should also consider removing the condition for
> > marking a client as failed if the rebalance timeout has reached while the
> > JoinGroup was not received, so that the semantics of session_timeout and
> > rebalance_timeout are totally separated: the former is only used to
> > determine if a consumer member of the group should be marked as failed
> and
> > kicked out of the group, and the latter is only used to determine the
> > longest time coordinator should wait for PREPARE_REBALANCE phase. In
> other
> > words if a member did not send the JoinGroup in time of the
> > rebalance_timeout, we still include it in the new generation of the group
> > and use its old subscription info to send to leader for assignment. Later
> > if the member came back with HeartBeat request, we can still follow the
> > normal path to bring it to the latest generation while checking that its
> > sent JoinGroup request contains the same subscription info as we used to
> > assign the partitions previously (which should be likely the case in
> > practice). In addition, we should let static members to not send the
> > LeaveGroup request when it is gracefully shutdown, so that a static
> member
> > can only be leaving the group if its session has timed out, OR it has
> been
> > indicated to not exist in the group any more (details below).
> >
> > 1.b) We have a parallel discussion about Incremental Cooperative
> > Rebalancing, in which we will encode the "when to rebalance" logic at the
> > application level, instead of at the protocol level. By doing this we can
> > also enable a few other optimizations, e.g. at the Streams level to first
> > build up the state store as standby tasks and then trigger a second
> > rebalance to actually migrate the active tasks while keeping the actual
> > rebalance latency and hence unavailability window to be small (
> >
> >
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907214421&amp;sdata=Etn5Y%2BVFTo0GQ1BydPVMUUYbEf04wyhqzKrs3OjCmC8%3D&amp;reserved=0
> ).
> > I'd propose we align
> > KIP-345 along with this idea, and hence do not add the expansion_timeout
> as
> > part of the protocol layer, but only do that at the application's
> > coordinator / assignor layer (Connect, Streams, etc). We can still,
> > deprecate the "*group.initial.rebalance.delay.ms
> > <
> >
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907370676&amp;sdata=fQfmBVBzFRr%2B4w2Fasv2aCzGxrbC97JTCdClEEvKWeQ%3D&amp;reserved=0
> >*"
> > though as part of this KIP
> > since we have discussed about its limit and think it is actually not a
> very
> > good design and could be replaced with client-side logic above.
> >
> >
> > 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> > specifically, let's say after we have upgraded broker version to be able
> to
> > recognize the new versions of JoinGroup request and the admin requests,
> how
> > should we upgrade the clients and enable static groups? On top of my head
> > if we do a rolling bounce in which we set the member.name config as well
> > as
> > optionally increase the session.timeout config when we bounce each
> > instance, then during this rolling bounces we will have a group contained
> > with both dynamic members and static members. It means that we should
> have
> > the group to allow such scenario (i.e. we cannot reject JoinGroup
> requests
> > from dynamic members), and hence the "member.name" -> "member.id"
> mapping
> > will only be partial at this scenario. Also could you describe if the
> > upgrade to the first version that support this feature would ever get any
> > benefits, or only the future upgrade path for rolling bounces could get
> > benefits out of this feature?
> >
> > If that's the case and we will do 1) as suggested above, do we still need
> > the enableStaticMembership and enableDynamicMembership admin requests any
> > more? Seems it is not necessary any more as we will only have the notion
> of
> > "dynamic or static members" that can co-exist in a group while there no
> > notion of "dynamic or static groups", and hence these two requests are
> not
> > needed anymore.
> >
> >
> > 3. We need to briefly talk about the implications for ACL as we introduce
> > new admin requests that are related to a specific group.id. For example,
> > we
> > need to make sure that whoever created the group or joined the group can
> > actually send admin requests for the group, otherwise the application
> > owners need to bother the Kafka operators on a multi-tenant cluster every
> > time they want to send any admin requests for their groups which would be
> > an operational nightmare.
> >
> >
> > 4. I like Jason's suggestion of adding an optional field for the list of
> > member names, and I'm wondering if that can be done as part of the
> > forceStaticRebalance request: i.e. by passing a list of members, we will
> > enforce a rebalance immediately since it indicates that some static
> member
> > will be officially kicked out of the group and some new static members
> may
> > be added. So back to 1.a) above, a static member can only be kicked out
> of
> > the group if a) its session (arguably long period of time) has timed out,
> > and b) this admin request explicitly state that it is no longer part of
> the
> > group. As for execution I'm fine with keeping it as a future work of this
> > KIP if you'd like to make its scope smaller.
> >
> > Following are minor comments:
> >
> > 5. I'm not sure if we need to include "member.name" as part of the
> > OffsetCommitRequest for fencing purposes, as I think the memberId plus
> the
> > generation number should be sufficient for fencing even with static
> > members.
> >
> > 6. As mentioned above, if we agree to do 1) we can get rid of the "
> > LEAVE_GROUP_ON_CLOSE_CONFIG" config.
> >
> >
> > Guozhang
> >
> >
> >
> >
> > On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
> >
> > > Hey Boyang,
> > >
> > > Thanks for the proposal! This is very useful. I have some comments
> below:
> > >
> > > 1) The motivation currently explicitly states that the goal is to
> improve
> > > performance for heavy state application. It seems that the motivation
> can
> > > be stronger with the following use-case. Currently for MirrorMaker
> > cluster
> > > with e.g. 100 MirrorMaker processes, it will take a long time to
> rolling
> > > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> > > will trigger a rebalance which currently pause the consumption of the
> all
> > > partitions of the MirrorMaker cluster. With the change stated in this
> > > patch, as long as a MirrorMaker can restart within the specified
> timeout
> > > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> > for
> > > leader restart) for the entire rolling bounce, which will significantly
> > > improves the availability of the MirrorMaker pipeline. In my opinion,
> the
> > > main benefit of the KIP is to avoid unnecessary rebalance if the
> consumer
> > > process can be restarted within soon, which helps performance even if
> > > overhead of state shuffling for a given process is small.
> > >
> > > 2) In order to simplify the KIP reading, can you follow the writeup
> style
> > > of other KIP (e.g. KIP-98) and list the interface change such as new
> > > configs (e.g. registration timeout), new request/response, new
> > AdminClient
> > > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some
> of
> > > these are specified in the Proposed Change section which makes it a bit
> > > inconvenient to understand the new interface that will be exposed to
> > user.
> > > Explanation of the current two-phase rebalance protocol probably can be
> > > moved out of public interface section.
> > >
> > > 3) There are currently two version of JoinGroupRequest in the KIP and
> > only
> > > one of them has field memberId. This seems confusing.
> > >
> > > 4) It is mentioned in the KIP that "An admin API to force rebalance
> could
> > > be helpful here, but we will make a call once we finished the major
> > > implementation". So this seems to be still an open question in the
> > current
> > > design. We probably want to agree on this before voting for the KIP.
> > >
> > > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > > specify the name of the config key and the default config value?
> Possible
> > > default values include empty string or null (similar to transaction.id
> > in
> > > producer config).
> > >
> > > 6) Regarding the use of the topic "static_member_map" to persist member
> > > name map, currently if consumer coordinator broker goes offline,
> > rebalance
> > > is triggered and consumers will try connect to the new coordinator. If
> > > these consumers can connect to the new coordinator within
> > > max.poll.interval.ms which by default is 5 minutes, given that broker
> > can
> > > use a deterministic algorithm to determine the partition -> member_name
> > > mapping, each consumer should get assigned the same set of partitions
> > > without requiring state shuffling. So it is not clear whether we have a
> > > strong use-case for this new logic. Can you help clarify what is the
> > > benefit of using topic "static_member_map" to persist member name map?
> > >
> > > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > > mentioned that "we are using expansion timeout to replace rebalance
> > > timeout, which is configured by max.poll.intervals from client side,
> and
> > > using registration timeout to replace session timeout". Currently the
> > > default max.poll.interval.ms is configured to be 5 minutes and there
> > will
> > > be only one rebalance if all new consumers can join within 5 minutes.
> So
> > it
> > > is not clear whether we have a strong use-case for this new config. Can
> > you
> > > explain what is the benefit of introducing this new config?
> > >
> > > 8) It is mentioned that "To distinguish between previous version of
> > > protocol, we will also increase the join group request version to v4
> when
> > > MEMBER_NAME is set" and "If the broker version is not the latest (<
> v4),
> > > the join group request shall be downgraded to v3 without setting the
> > member
> > > Id". It is probably simpler to just say that this feature is enabled if
> > > JoinGroupRequest V4 is supported on both client and broker and
> > MEMBER_NAME
> > > is configured with non-empty string.
> > >
> > > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET
> error
> > > in OffsetCommitResponse for "commit requests under static membership".
> > Can
> > > you clarify how broker determines whether the commit request is under
> > > static membership?
> > >
> > > Thanks,
> > > Dong
> > >
> >
> >
> > --
> > -- Guozhang
> >
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Konstantine Karantasis <ko...@confluent.io>.
Hi Boyang.

Thanks for preparing this KIP! It is making good progress and will be a
great improvement for stateful Kafka applications.

Apologies for my late reply, I was away for a while. Lots of great comments
so far, so I'll probably second most of them in what I suggest below at
this point.

When I first read the KIP, I wanted to start at the end with something that
wasn't highlighted a lot. That was the topic related to handling duplicate
members. I see now that the initial suggestion of handling this situation
during offset commit has been removed, and I agree with that. Issues
related to membership seem to be handled better when the member joins the
group rather than when it tries to commit offsets. This also simplifies how
many request types need to change in order to incorporate the new member
name field.

I also agree with what Jason and Guozhang have said regarding timeouts.
Although semantically, it's easier to think of every operation having its
own timeout, operationally this can become a burden. Thus, consolidation
seems preferable here. The definition of embedded protocols on top of the
base group membership protocol for rebalancing gives enough flexibility to
address such needs in each client component separately.

Finally, some minor comments:
In a few places the new/proposed changes are referred to as "current".
Which is a bit confusing considering that there is a protocol in place
already, and by "current" someone might understand the existing one. I'd
recommend using new/proposed or equivalent when referring to changes
introduced with KIP-345 and current/existing or equivalent when referring
to existing behavior.

There's the following sentence in the "Public Interfaces" section:
"Since for many stateful consumer/stream applications, the state shuffling
is more painful than short time partial unavailability."
However, my understanding is that the changes proposed with KIP-345 will
not exploit any partial availability. A suggestion for dealing with
temporary imbalances has been made in "Incremental Cooperative Rebalancing"
which can work well with KIP-345, but here I don't see proposed changes
that suggest that some resources (e.g. partitions) will keep being used
while others will not be utilized. Thus, you might want to adjust this
sentence. Correct me if I'm missing something related to that.

In the rejected alternatives, under point 2) I read "we can copy the member
id to the config files". I believe it means to say "member name" unless I'm
missing something about reusing member ids. Also below I read: "By allowing
consumers to optionally specifying a member id" which probably implies
"member name" again. In a sense this section highlights a potential
confusion between member name and member id. I wonder if we could come up
with a better term for the new field. StaticTag, StaticLabel, or even
StaticName are some suggestions that could potentially help with confusion
between MemberId and MemberName and what corresponds to what. But I
wouldn't like to disrupt the discussion with naming conventions too much at
this point. I just mention it here as a thought.

Looking forward to see the final details of this KIP. Great work so far!

Konstantine


On Tue, Nov 20, 2018 at 4:23 AM Boyang Chen <bc...@outlook.com> wrote:

> Thanks Guozhang for the great summary here, and I have been following up
> the action items here.
>
>
>   1.  I already updated the KIP to remove the expansion timeout and
> registration timeout. Great to see them being addressed in client side!
>   2.  I double checked the design and I believe that it is ok to have both
> static member and dynamic member co-exist in the same group. So the upgrade
> shouldn't be destructive and we are removing the two membership protocol
> switching APIs.
>   3.  I only have question about this one. I'm still reading the KafkaApis
> code here. Should I just use the same authorization logic for
> ForceStaticRebalanceRequest as JoinGroupRequest?
>   4.  I'm very excited to see this work with K8! Like you suggested, this
> feature could be better addressed in a separate KIP because it is pretty
> independent. I could start drafting the KIP once the current proposal is
> approved.
>   5.  I believe that we don't need fencing in offset commit request, since
> duplicate member.name issue could be handled by join group request. We
> shall reject join group with known member name but no member id (which
> means we already have an active member using this identity).
>   6.  I agree to remove that internal config once we move forward with
> static membership. And I already removed the entire section from the KIP.
>
> Let me know if you have other concerns.
>
> Best,
> Boyang
> ________________________________
> From: Guozhang Wang <wa...@gmail.com>
> Sent: Tuesday, November 20, 2018 4:21 PM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hello Boyang,
>
> Thanks a lot for the KIP! It is a great write-up and I appreciate your
> patience answering to the feedbacks from the community. I'd like to add my
> 2cents here:
>
> 1. By introducing another two timeout configs, registration_timeout and
> expansion_timeout, we are effectively having four timeout configs: session
> timeout, rebalance timeout (configured as "max.poll.interval.ms" on client
> side), and these two. Interplaying these timeout configs can be quite hard
> for users with such complexity, and hence I'm wondering if we can simplify
> the situation with as less possible timeout configs as possible. Here is a
> concrete suggestion I'd like propose:
>
> 1.a) Instead of introducing a registration_timeout in addition to the
> session_timeout for static members, we can just reuse the session_timeout
> and ask users to set it to a larger value when they are upgrading a dynamic
> client to a static client by setting the "member.name" at the same time.
> By
> default, the broker-side min.session.timeout is 6 seconds and
> max.session.timeout is 5 minutes, which seems reasonable to me (we can of
> course modify this broker config to enlarge the valid interval if we want
> in practice). And then we should also consider removing the condition for
> marking a client as failed if the rebalance timeout has reached while the
> JoinGroup was not received, so that the semantics of session_timeout and
> rebalance_timeout are totally separated: the former is only used to
> determine if a consumer member of the group should be marked as failed and
> kicked out of the group, and the latter is only used to determine the
> longest time coordinator should wait for PREPARE_REBALANCE phase. In other
> words if a member did not send the JoinGroup in time of the
> rebalance_timeout, we still include it in the new generation of the group
> and use its old subscription info to send to leader for assignment. Later
> if the member came back with HeartBeat request, we can still follow the
> normal path to bring it to the latest generation while checking that its
> sent JoinGroup request contains the same subscription info as we used to
> assign the partitions previously (which should be likely the case in
> practice). In addition, we should let static members to not send the
> LeaveGroup request when it is gracefully shutdown, so that a static member
> can only be leaving the group if its session has timed out, OR it has been
> indicated to not exist in the group any more (details below).
>
> 1.b) We have a parallel discussion about Incremental Cooperative
> Rebalancing, in which we will encode the "when to rebalance" logic at the
> application level, instead of at the protocol level. By doing this we can
> also enable a few other optimizations, e.g. at the Streams level to first
> build up the state store as standby tasks and then trigger a second
> rebalance to actually migrate the active tasks while keeping the actual
> rebalance latency and hence unavailability window to be small (
>
> https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907214421&amp;sdata=Etn5Y%2BVFTo0GQ1BydPVMUUYbEf04wyhqzKrs3OjCmC8%3D&amp;reserved=0).
> I'd propose we align
> KIP-345 along with this idea, and hence do not add the expansion_timeout as
> part of the protocol layer, but only do that at the application's
> coordinator / assignor layer (Connect, Streams, etc). We can still,
> deprecate the "*group.initial.rebalance.delay.ms
> <
> https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907370676&amp;sdata=fQfmBVBzFRr%2B4w2Fasv2aCzGxrbC97JTCdClEEvKWeQ%3D&amp;reserved=0>*"
> though as part of this KIP
> since we have discussed about its limit and think it is actually not a very
> good design and could be replaced with client-side logic above.
>
>
> 2. I'd like to see your thoughts on the upgrade path for this KIP. More
> specifically, let's say after we have upgraded broker version to be able to
> recognize the new versions of JoinGroup request and the admin requests, how
> should we upgrade the clients and enable static groups? On top of my head
> if we do a rolling bounce in which we set the member.name config as well
> as
> optionally increase the session.timeout config when we bounce each
> instance, then during this rolling bounces we will have a group contained
> with both dynamic members and static members. It means that we should have
> the group to allow such scenario (i.e. we cannot reject JoinGroup requests
> from dynamic members), and hence the "member.name" -> "member.id" mapping
> will only be partial at this scenario. Also could you describe if the
> upgrade to the first version that support this feature would ever get any
> benefits, or only the future upgrade path for rolling bounces could get
> benefits out of this feature?
>
> If that's the case and we will do 1) as suggested above, do we still need
> the enableStaticMembership and enableDynamicMembership admin requests any
> more? Seems it is not necessary any more as we will only have the notion of
> "dynamic or static members" that can co-exist in a group while there no
> notion of "dynamic or static groups", and hence these two requests are not
> needed anymore.
>
>
> 3. We need to briefly talk about the implications for ACL as we introduce
> new admin requests that are related to a specific group.id. For example,
> we
> need to make sure that whoever created the group or joined the group can
> actually send admin requests for the group, otherwise the application
> owners need to bother the Kafka operators on a multi-tenant cluster every
> time they want to send any admin requests for their groups which would be
> an operational nightmare.
>
>
> 4. I like Jason's suggestion of adding an optional field for the list of
> member names, and I'm wondering if that can be done as part of the
> forceStaticRebalance request: i.e. by passing a list of members, we will
> enforce a rebalance immediately since it indicates that some static member
> will be officially kicked out of the group and some new static members may
> be added. So back to 1.a) above, a static member can only be kicked out of
> the group if a) its session (arguably long period of time) has timed out,
> and b) this admin request explicitly state that it is no longer part of the
> group. As for execution I'm fine with keeping it as a future work of this
> KIP if you'd like to make its scope smaller.
>
> Following are minor comments:
>
> 5. I'm not sure if we need to include "member.name" as part of the
> OffsetCommitRequest for fencing purposes, as I think the memberId plus the
> generation number should be sufficient for fencing even with static
> members.
>
> 6. As mentioned above, if we agree to do 1) we can get rid of the "
> LEAVE_GROUP_ON_CLOSE_CONFIG" config.
>
>
> Guozhang
>
>
>
>
> On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:
>
> > Hey Boyang,
> >
> > Thanks for the proposal! This is very useful. I have some comments below:
> >
> > 1) The motivation currently explicitly states that the goal is to improve
> > performance for heavy state application. It seems that the motivation can
> > be stronger with the following use-case. Currently for MirrorMaker
> cluster
> > with e.g. 100 MirrorMaker processes, it will take a long time to rolling
> > bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> > will trigger a rebalance which currently pause the consumption of the all
> > partitions of the MirrorMaker cluster. With the change stated in this
> > patch, as long as a MirrorMaker can restart within the specified timeout
> > (e.g. 2 minutes), then we only need constant number of rebalance (e.g.
> for
> > leader restart) for the entire rolling bounce, which will significantly
> > improves the availability of the MirrorMaker pipeline. In my opinion, the
> > main benefit of the KIP is to avoid unnecessary rebalance if the consumer
> > process can be restarted within soon, which helps performance even if
> > overhead of state shuffling for a given process is small.
> >
> > 2) In order to simplify the KIP reading, can you follow the writeup style
> > of other KIP (e.g. KIP-98) and list the interface change such as new
> > configs (e.g. registration timeout), new request/response, new
> AdminClient
> > API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> > these are specified in the Proposed Change section which makes it a bit
> > inconvenient to understand the new interface that will be exposed to
> user.
> > Explanation of the current two-phase rebalance protocol probably can be
> > moved out of public interface section.
> >
> > 3) There are currently two version of JoinGroupRequest in the KIP and
> only
> > one of them has field memberId. This seems confusing.
> >
> > 4) It is mentioned in the KIP that "An admin API to force rebalance could
> > be helpful here, but we will make a call once we finished the major
> > implementation". So this seems to be still an open question in the
> current
> > design. We probably want to agree on this before voting for the KIP.
> >
> > 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> > specify the name of the config key and the default config value? Possible
> > default values include empty string or null (similar to transaction.id
> in
> > producer config).
> >
> > 6) Regarding the use of the topic "static_member_map" to persist member
> > name map, currently if consumer coordinator broker goes offline,
> rebalance
> > is triggered and consumers will try connect to the new coordinator. If
> > these consumers can connect to the new coordinator within
> > max.poll.interval.ms which by default is 5 minutes, given that broker
> can
> > use a deterministic algorithm to determine the partition -> member_name
> > mapping, each consumer should get assigned the same set of partitions
> > without requiring state shuffling. So it is not clear whether we have a
> > strong use-case for this new logic. Can you help clarify what is the
> > benefit of using topic "static_member_map" to persist member name map?
> >
> > 7) Regarding the introduction of the expensionTimeoutMs config, it is
> > mentioned that "we are using expansion timeout to replace rebalance
> > timeout, which is configured by max.poll.intervals from client side, and
> > using registration timeout to replace session timeout". Currently the
> > default max.poll.interval.ms is configured to be 5 minutes and there
> will
> > be only one rebalance if all new consumers can join within 5 minutes. So
> it
> > is not clear whether we have a strong use-case for this new config. Can
> you
> > explain what is the benefit of introducing this new config?
> >
> > 8) It is mentioned that "To distinguish between previous version of
> > protocol, we will also increase the join group request version to v4 when
> > MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> > the join group request shall be downgraded to v3 without setting the
> member
> > Id". It is probably simpler to just say that this feature is enabled if
> > JoinGroupRequest V4 is supported on both client and broker and
> MEMBER_NAME
> > is configured with non-empty string.
> >
> > 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> > in OffsetCommitResponse for "commit requests under static membership".
> Can
> > you clarify how broker determines whether the commit request is under
> > static membership?
> >
> > Thanks,
> > Dong
> >
>
>
> --
> -- Guozhang
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Guozhang for the great summary here, and I have been following up the action items here.


  1.  I already updated the KIP to remove the expansion timeout and registration timeout. Great to see them being addressed in client side!
  2.  I double checked the design and I believe that it is ok to have both static member and dynamic member co-exist in the same group. So the upgrade shouldn't be destructive and we are removing the two membership protocol switching APIs.
  3.  I only have question about this one. I'm still reading the KafkaApis code here. Should I just use the same authorization logic for ForceStaticRebalanceRequest as JoinGroupRequest?
  4.  I'm very excited to see this work with K8! Like you suggested, this feature could be better addressed in a separate KIP because it is pretty independent. I could start drafting the KIP once the current proposal is approved.
  5.  I believe that we don't need fencing in offset commit request, since duplicate member.name issue could be handled by join group request. We shall reject join group with known member name but no member id (which means we already have an active member using this identity).
  6.  I agree to remove that internal config once we move forward with static membership. And I already removed the entire section from the KIP.

Let me know if you have other concerns.

Best,
Boyang
________________________________
From: Guozhang Wang <wa...@gmail.com>
Sent: Tuesday, November 20, 2018 4:21 PM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hello Boyang,

Thanks a lot for the KIP! It is a great write-up and I appreciate your
patience answering to the feedbacks from the community. I'd like to add my
2cents here:

1. By introducing another two timeout configs, registration_timeout and
expansion_timeout, we are effectively having four timeout configs: session
timeout, rebalance timeout (configured as "max.poll.interval.ms" on client
side), and these two. Interplaying these timeout configs can be quite hard
for users with such complexity, and hence I'm wondering if we can simplify
the situation with as less possible timeout configs as possible. Here is a
concrete suggestion I'd like propose:

1.a) Instead of introducing a registration_timeout in addition to the
session_timeout for static members, we can just reuse the session_timeout
and ask users to set it to a larger value when they are upgrading a dynamic
client to a static client by setting the "member.name" at the same time. By
default, the broker-side min.session.timeout is 6 seconds and
max.session.timeout is 5 minutes, which seems reasonable to me (we can of
course modify this broker config to enlarge the valid interval if we want
in practice). And then we should also consider removing the condition for
marking a client as failed if the rebalance timeout has reached while the
JoinGroup was not received, so that the semantics of session_timeout and
rebalance_timeout are totally separated: the former is only used to
determine if a consumer member of the group should be marked as failed and
kicked out of the group, and the latter is only used to determine the
longest time coordinator should wait for PREPARE_REBALANCE phase. In other
words if a member did not send the JoinGroup in time of the
rebalance_timeout, we still include it in the new generation of the group
and use its old subscription info to send to leader for assignment. Later
if the member came back with HeartBeat request, we can still follow the
normal path to bring it to the latest generation while checking that its
sent JoinGroup request contains the same subscription info as we used to
assign the partitions previously (which should be likely the case in
practice). In addition, we should let static members to not send the
LeaveGroup request when it is gracefully shutdown, so that a static member
can only be leaving the group if its session has timed out, OR it has been
indicated to not exist in the group any more (details below).

1.b) We have a parallel discussion about Incremental Cooperative
Rebalancing, in which we will encode the "when to rebalance" logic at the
application level, instead of at the protocol level. By doing this we can
also enable a few other optimizations, e.g. at the Streams level to first
build up the state store as standby tasks and then trigger a second
rebalance to actually migrate the active tasks while keeping the actual
rebalance latency and hence unavailability window to be small (
https://eur01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fissues.apache.org%2Fjira%2Fbrowse%2FKAFKA-6145&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907214421&amp;sdata=Etn5Y%2BVFTo0GQ1BydPVMUUYbEf04wyhqzKrs3OjCmC8%3D&amp;reserved=0). I'd propose we align
KIP-345 along with this idea, and hence do not add the expansion_timeout as
part of the protocol layer, but only do that at the application's
coordinator / assignor layer (Connect, Streams, etc). We can still,
deprecate the "*group.initial.rebalance.delay.ms
<https://eur01.safelinks.protection.outlook.com/?url=http%3A%2F%2Fgroup.initial.rebalance.delay.ms&amp;data=02%7C01%7C%7C7272fe45059f4afa205208d64ec12c1e%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636782988907370676&amp;sdata=fQfmBVBzFRr%2B4w2Fasv2aCzGxrbC97JTCdClEEvKWeQ%3D&amp;reserved=0>*" though as part of this KIP
since we have discussed about its limit and think it is actually not a very
good design and could be replaced with client-side logic above.


2. I'd like to see your thoughts on the upgrade path for this KIP. More
specifically, let's say after we have upgraded broker version to be able to
recognize the new versions of JoinGroup request and the admin requests, how
should we upgrade the clients and enable static groups? On top of my head
if we do a rolling bounce in which we set the member.name config as well as
optionally increase the session.timeout config when we bounce each
instance, then during this rolling bounces we will have a group contained
with both dynamic members and static members. It means that we should have
the group to allow such scenario (i.e. we cannot reject JoinGroup requests
from dynamic members), and hence the "member.name" -> "member.id" mapping
will only be partial at this scenario. Also could you describe if the
upgrade to the first version that support this feature would ever get any
benefits, or only the future upgrade path for rolling bounces could get
benefits out of this feature?

If that's the case and we will do 1) as suggested above, do we still need
the enableStaticMembership and enableDynamicMembership admin requests any
more? Seems it is not necessary any more as we will only have the notion of
"dynamic or static members" that can co-exist in a group while there no
notion of "dynamic or static groups", and hence these two requests are not
needed anymore.


3. We need to briefly talk about the implications for ACL as we introduce
new admin requests that are related to a specific group.id. For example, we
need to make sure that whoever created the group or joined the group can
actually send admin requests for the group, otherwise the application
owners need to bother the Kafka operators on a multi-tenant cluster every
time they want to send any admin requests for their groups which would be
an operational nightmare.


4. I like Jason's suggestion of adding an optional field for the list of
member names, and I'm wondering if that can be done as part of the
forceStaticRebalance request: i.e. by passing a list of members, we will
enforce a rebalance immediately since it indicates that some static member
will be officially kicked out of the group and some new static members may
be added. So back to 1.a) above, a static member can only be kicked out of
the group if a) its session (arguably long period of time) has timed out,
and b) this admin request explicitly state that it is no longer part of the
group. As for execution I'm fine with keeping it as a future work of this
KIP if you'd like to make its scope smaller.

Following are minor comments:

5. I'm not sure if we need to include "member.name" as part of the
OffsetCommitRequest for fencing purposes, as I think the memberId plus the
generation number should be sufficient for fencing even with static members.

6. As mentioned above, if we agree to do 1) we can get rid of the "
LEAVE_GROUP_ON_CLOSE_CONFIG" config.


Guozhang




On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:

> Hey Boyang,
>
> Thanks for the proposal! This is very useful. I have some comments below:
>
> 1) The motivation currently explicitly states that the goal is to improve
> performance for heavy state application. It seems that the motivation can
> be stronger with the following use-case. Currently for MirrorMaker cluster
> with e.g. 100 MirrorMaker processes, it will take a long time to rolling
> bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> will trigger a rebalance which currently pause the consumption of the all
> partitions of the MirrorMaker cluster. With the change stated in this
> patch, as long as a MirrorMaker can restart within the specified timeout
> (e.g. 2 minutes), then we only need constant number of rebalance (e.g. for
> leader restart) for the entire rolling bounce, which will significantly
> improves the availability of the MirrorMaker pipeline. In my opinion, the
> main benefit of the KIP is to avoid unnecessary rebalance if the consumer
> process can be restarted within soon, which helps performance even if
> overhead of state shuffling for a given process is small.
>
> 2) In order to simplify the KIP reading, can you follow the writeup style
> of other KIP (e.g. KIP-98) and list the interface change such as new
> configs (e.g. registration timeout), new request/response, new AdminClient
> API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> these are specified in the Proposed Change section which makes it a bit
> inconvenient to understand the new interface that will be exposed to user.
> Explanation of the current two-phase rebalance protocol probably can be
> moved out of public interface section.
>
> 3) There are currently two version of JoinGroupRequest in the KIP and only
> one of them has field memberId. This seems confusing.
>
> 4) It is mentioned in the KIP that "An admin API to force rebalance could
> be helpful here, but we will make a call once we finished the major
> implementation". So this seems to be still an open question in the current
> design. We probably want to agree on this before voting for the KIP.
>
> 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> specify the name of the config key and the default config value? Possible
> default values include empty string or null (similar to transaction.id in
> producer config).
>
> 6) Regarding the use of the topic "static_member_map" to persist member
> name map, currently if consumer coordinator broker goes offline, rebalance
> is triggered and consumers will try connect to the new coordinator. If
> these consumers can connect to the new coordinator within
> max.poll.interval.ms which by default is 5 minutes, given that broker can
> use a deterministic algorithm to determine the partition -> member_name
> mapping, each consumer should get assigned the same set of partitions
> without requiring state shuffling. So it is not clear whether we have a
> strong use-case for this new logic. Can you help clarify what is the
> benefit of using topic "static_member_map" to persist member name map?
>
> 7) Regarding the introduction of the expensionTimeoutMs config, it is
> mentioned that "we are using expansion timeout to replace rebalance
> timeout, which is configured by max.poll.intervals from client side, and
> using registration timeout to replace session timeout". Currently the
> default max.poll.interval.ms is configured to be 5 minutes and there will
> be only one rebalance if all new consumers can join within 5 minutes. So it
> is not clear whether we have a strong use-case for this new config. Can you
> explain what is the benefit of introducing this new config?
>
> 8) It is mentioned that "To distinguish between previous version of
> protocol, we will also increase the join group request version to v4 when
> MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> the join group request shall be downgraded to v3 without setting the member
> Id". It is probably simpler to just say that this feature is enabled if
> JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
> is configured with non-empty string.
>
> 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> in OffsetCommitResponse for "commit requests under static membership". Can
> you clarify how broker determines whether the commit request is under
> static membership?
>
> Thanks,
> Dong
>


--
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Guozhang Wang <wa...@gmail.com>.
Hello Boyang,

Thanks a lot for the KIP! It is a great write-up and I appreciate your
patience answering to the feedbacks from the community. I'd like to add my
2cents here:

1. By introducing another two timeout configs, registration_timeout and
expansion_timeout, we are effectively having four timeout configs: session
timeout, rebalance timeout (configured as "max.poll.interval.ms" on client
side), and these two. Interplaying these timeout configs can be quite hard
for users with such complexity, and hence I'm wondering if we can simplify
the situation with as less possible timeout configs as possible. Here is a
concrete suggestion I'd like propose:

1.a) Instead of introducing a registration_timeout in addition to the
session_timeout for static members, we can just reuse the session_timeout
and ask users to set it to a larger value when they are upgrading a dynamic
client to a static client by setting the "member.name" at the same time. By
default, the broker-side min.session.timeout is 6 seconds and
max.session.timeout is 5 minutes, which seems reasonable to me (we can of
course modify this broker config to enlarge the valid interval if we want
in practice). And then we should also consider removing the condition for
marking a client as failed if the rebalance timeout has reached while the
JoinGroup was not received, so that the semantics of session_timeout and
rebalance_timeout are totally separated: the former is only used to
determine if a consumer member of the group should be marked as failed and
kicked out of the group, and the latter is only used to determine the
longest time coordinator should wait for PREPARE_REBALANCE phase. In other
words if a member did not send the JoinGroup in time of the
rebalance_timeout, we still include it in the new generation of the group
and use its old subscription info to send to leader for assignment. Later
if the member came back with HeartBeat request, we can still follow the
normal path to bring it to the latest generation while checking that its
sent JoinGroup request contains the same subscription info as we used to
assign the partitions previously (which should be likely the case in
practice). In addition, we should let static members to not send the
LeaveGroup request when it is gracefully shutdown, so that a static member
can only be leaving the group if its session has timed out, OR it has been
indicated to not exist in the group any more (details below).

1.b) We have a parallel discussion about Incremental Cooperative
Rebalancing, in which we will encode the "when to rebalance" logic at the
application level, instead of at the protocol level. By doing this we can
also enable a few other optimizations, e.g. at the Streams level to first
build up the state store as standby tasks and then trigger a second
rebalance to actually migrate the active tasks while keeping the actual
rebalance latency and hence unavailability window to be small (
https://issues.apache.org/jira/browse/KAFKA-6145). I'd propose we align
KIP-345 along with this idea, and hence do not add the expansion_timeout as
part of the protocol layer, but only do that at the application's
coordinator / assignor layer (Connect, Streams, etc). We can still,
deprecate the "*group.initial.rebalance.delay.ms
<http://group.initial.rebalance.delay.ms>*" though as part of this KIP
since we have discussed about its limit and think it is actually not a very
good design and could be replaced with client-side logic above.


2. I'd like to see your thoughts on the upgrade path for this KIP. More
specifically, let's say after we have upgraded broker version to be able to
recognize the new versions of JoinGroup request and the admin requests, how
should we upgrade the clients and enable static groups? On top of my head
if we do a rolling bounce in which we set the member.name config as well as
optionally increase the session.timeout config when we bounce each
instance, then during this rolling bounces we will have a group contained
with both dynamic members and static members. It means that we should have
the group to allow such scenario (i.e. we cannot reject JoinGroup requests
from dynamic members), and hence the "member.name" -> "member.id" mapping
will only be partial at this scenario. Also could you describe if the
upgrade to the first version that support this feature would ever get any
benefits, or only the future upgrade path for rolling bounces could get
benefits out of this feature?

If that's the case and we will do 1) as suggested above, do we still need
the enableStaticMembership and enableDynamicMembership admin requests any
more? Seems it is not necessary any more as we will only have the notion of
"dynamic or static members" that can co-exist in a group while there no
notion of "dynamic or static groups", and hence these two requests are not
needed anymore.


3. We need to briefly talk about the implications for ACL as we introduce
new admin requests that are related to a specific group.id. For example, we
need to make sure that whoever created the group or joined the group can
actually send admin requests for the group, otherwise the application
owners need to bother the Kafka operators on a multi-tenant cluster every
time they want to send any admin requests for their groups which would be
an operational nightmare.


4. I like Jason's suggestion of adding an optional field for the list of
member names, and I'm wondering if that can be done as part of the
forceStaticRebalance request: i.e. by passing a list of members, we will
enforce a rebalance immediately since it indicates that some static member
will be officially kicked out of the group and some new static members may
be added. So back to 1.a) above, a static member can only be kicked out of
the group if a) its session (arguably long period of time) has timed out,
and b) this admin request explicitly state that it is no longer part of the
group. As for execution I'm fine with keeping it as a future work of this
KIP if you'd like to make its scope smaller.

Following are minor comments:

5. I'm not sure if we need to include "member.name" as part of the
OffsetCommitRequest for fencing purposes, as I think the memberId plus the
generation number should be sufficient for fencing even with static members.

6. As mentioned above, if we agree to do 1) we can get rid of the "
LEAVE_GROUP_ON_CLOSE_CONFIG" config.


Guozhang




On Sat, Nov 17, 2018 at 5:53 PM Dong Lin <li...@gmail.com> wrote:

> Hey Boyang,
>
> Thanks for the proposal! This is very useful. I have some comments below:
>
> 1) The motivation currently explicitly states that the goal is to improve
> performance for heavy state application. It seems that the motivation can
> be stronger with the following use-case. Currently for MirrorMaker cluster
> with e.g. 100 MirrorMaker processes, it will take a long time to rolling
> bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
> will trigger a rebalance which currently pause the consumption of the all
> partitions of the MirrorMaker cluster. With the change stated in this
> patch, as long as a MirrorMaker can restart within the specified timeout
> (e.g. 2 minutes), then we only need constant number of rebalance (e.g. for
> leader restart) for the entire rolling bounce, which will significantly
> improves the availability of the MirrorMaker pipeline. In my opinion, the
> main benefit of the KIP is to avoid unnecessary rebalance if the consumer
> process can be restarted within soon, which helps performance even if
> overhead of state shuffling for a given process is small.
>
> 2) In order to simplify the KIP reading, can you follow the writeup style
> of other KIP (e.g. KIP-98) and list the interface change such as new
> configs (e.g. registration timeout), new request/response, new AdminClient
> API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
> these are specified in the Proposed Change section which makes it a bit
> inconvenient to understand the new interface that will be exposed to user.
> Explanation of the current two-phase rebalance protocol probably can be
> moved out of public interface section.
>
> 3) There are currently two version of JoinGroupRequest in the KIP and only
> one of them has field memberId. This seems confusing.
>
> 4) It is mentioned in the KIP that "An admin API to force rebalance could
> be helpful here, but we will make a call once we finished the major
> implementation". So this seems to be still an open question in the current
> design. We probably want to agree on this before voting for the KIP.
>
> 5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
> specify the name of the config key and the default config value? Possible
> default values include empty string or null (similar to transaction.id in
> producer config).
>
> 6) Regarding the use of the topic "static_member_map" to persist member
> name map, currently if consumer coordinator broker goes offline, rebalance
> is triggered and consumers will try connect to the new coordinator. If
> these consumers can connect to the new coordinator within
> max.poll.interval.ms which by default is 5 minutes, given that broker can
> use a deterministic algorithm to determine the partition -> member_name
> mapping, each consumer should get assigned the same set of partitions
> without requiring state shuffling. So it is not clear whether we have a
> strong use-case for this new logic. Can you help clarify what is the
> benefit of using topic "static_member_map" to persist member name map?
>
> 7) Regarding the introduction of the expensionTimeoutMs config, it is
> mentioned that "we are using expansion timeout to replace rebalance
> timeout, which is configured by max.poll.intervals from client side, and
> using registration timeout to replace session timeout". Currently the
> default max.poll.interval.ms is configured to be 5 minutes and there will
> be only one rebalance if all new consumers can join within 5 minutes. So it
> is not clear whether we have a strong use-case for this new config. Can you
> explain what is the benefit of introducing this new config?
>
> 8) It is mentioned that "To distinguish between previous version of
> protocol, we will also increase the join group request version to v4 when
> MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
> the join group request shall be downgraded to v3 without setting the member
> Id". It is probably simpler to just say that this feature is enabled if
> JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
> is configured with non-empty string.
>
> 9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
> in OffsetCommitResponse for "commit requests under static membership". Can
> you clarify how broker determines whether the commit request is under
> static membership?
>
> Thanks,
> Dong
>


-- 
-- Guozhang

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Dong Lin <li...@gmail.com>.
Hey Boyang,

Thanks for the proposal! This is very useful. I have some comments below:

1) The motivation currently explicitly states that the goal is to improve
performance for heavy state application. It seems that the motivation can
be stronger with the following use-case. Currently for MirrorMaker cluster
with e.g. 100 MirrorMaker processes, it will take a long time to rolling
bounce the entire MirrorMaker cluster. Each MirrorMaker process restart
will trigger a rebalance which currently pause the consumption of the all
partitions of the MirrorMaker cluster. With the change stated in this
patch, as long as a MirrorMaker can restart within the specified timeout
(e.g. 2 minutes), then we only need constant number of rebalance (e.g. for
leader restart) for the entire rolling bounce, which will significantly
improves the availability of the MirrorMaker pipeline. In my opinion, the
main benefit of the KIP is to avoid unnecessary rebalance if the consumer
process can be restarted within soon, which helps performance even if
overhead of state shuffling for a given process is small.

2) In order to simplify the KIP reading, can you follow the writeup style
of other KIP (e.g. KIP-98) and list the interface change such as new
configs (e.g. registration timeout), new request/response, new AdminClient
API and new error code (e.g. DUPLICATE_STATIC_MEMBER)? Currently some of
these are specified in the Proposed Change section which makes it a bit
inconvenient to understand the new interface that will be exposed to user.
Explanation of the current two-phase rebalance protocol probably can be
moved out of public interface section.

3) There are currently two version of JoinGroupRequest in the KIP and only
one of them has field memberId. This seems confusing.

4) It is mentioned in the KIP that "An admin API to force rebalance could
be helpful here, but we will make a call once we finished the major
implementation". So this seems to be still an open question in the current
design. We probably want to agree on this before voting for the KIP.

5) The KIP currently adds new config MEMBER_NAME for consumer. Can you
specify the name of the config key and the default config value? Possible
default values include empty string or null (similar to transaction.id in
producer config).

6) Regarding the use of the topic "static_member_map" to persist member
name map, currently if consumer coordinator broker goes offline, rebalance
is triggered and consumers will try connect to the new coordinator. If
these consumers can connect to the new coordinator within
max.poll.interval.ms which by default is 5 minutes, given that broker can
use a deterministic algorithm to determine the partition -> member_name
mapping, each consumer should get assigned the same set of partitions
without requiring state shuffling. So it is not clear whether we have a
strong use-case for this new logic. Can you help clarify what is the
benefit of using topic "static_member_map" to persist member name map?

7) Regarding the introduction of the expensionTimeoutMs config, it is
mentioned that "we are using expansion timeout to replace rebalance
timeout, which is configured by max.poll.intervals from client side, and
using registration timeout to replace session timeout". Currently the
default max.poll.interval.ms is configured to be 5 minutes and there will
be only one rebalance if all new consumers can join within 5 minutes. So it
is not clear whether we have a strong use-case for this new config. Can you
explain what is the benefit of introducing this new config?

8) It is mentioned that "To distinguish between previous version of
protocol, we will also increase the join group request version to v4 when
MEMBER_NAME is set" and "If the broker version is not the latest (< v4),
the join group request shall be downgraded to v3 without setting the member
Id". It is probably simpler to just say that this feature is enabled if
JoinGroupRequest V4 is supported on both client and broker and MEMBER_NAME
is configured with non-empty string.

9) It is mentioned that broker may return NO_STATIC_MEMBER_INFO_SET error
in OffsetCommitResponse for "commit requests under static membership". Can
you clarify how broker determines whether the commit request is under
static membership?

Thanks,
Dong

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Colin and Jason for the further inputs!

I understand that k8 integration will be the long term solution for KStream to serve as a streaming platform, and I buy in the idea of "pre-registration" which could save us from using timeout tricks.


> If we have a way to indicate the
> expected group members, then the group can respond to a change much more
> quickly. There would be no need to wait 5 minutes for all members to join
> and it would be robust in the presence of failures. Ironically, static
> membership in this case makes the group more dynamic ;).

The caveat here is that existing KStream users are probably not easy to migrate to K8 or config other management tools immediately (including us), so in the static membership design I always try to minimize the amount of operation work needed by the client, hoping the user just needs to provide unique id generation and two timeout configs to easy onboard. This way it would be much easier to onboard static membership for all KStream users, not only benefiting K8 equipped users. The expected member list would be nice to have, which I would definitely like to discuss in a follow-up KIP. Do you think this makes sense?

> I think both of these issues could be solved by having some broker-side metadata about groups which is configured through the admin client.  If there was an "expected group size," stored on the broker-side, then we could rebalance immediately whenever the group size reached that size.  Otherwise, we could apply the rebalance delay, like now.  This would give lower latency when setting things up.

Setting expected group size in administrative way is a good suggestion, however we need to realize that in daily operation, we could always have transit failures or instance unavailable (AWS EC2 spin up failed) which invalidates our expectation. For example we are scaling up from 4 to 8 hosts, we would expect following operations:

  1.  Use admin tool to set group size from 4 to 8
  2.  Spin up new hosts...
  3.  Group will rebalance when we hit 8 join group requests

If in step 2 we failed to spin up one single host, the scale up mechanism will fail and get stuck until we fixed the problem by forcing rebalance the fleet. To fast scale up, 7 hosts is still better than 4 hosts IMO, and without expansion timeout we are paying extra loading time and human monitoring effort to trigger rebalance if necessary (we are introducing a forceStaticRebalance API here). So expansion timeout and registration timeout are useful to reduce mental burden when one operates the member group, which should be helpful.

> Expected group size is just an expectation, so the group would be allowed to get bigger than that.  We could also have another number which was the > maximum group size.  This really would be a hard upper limit on the size of the group, which admins could optionally configure.

The maximum group size is discussed in this JIRA<https://issues.apache.org/jira/browse/KAFKA-7641>. It is also useful for dynamic membership, thanks for the suggestion!

> When a new client joined a group, the server could send back a unique random 64-bit member ID.  The client could hold on to this ID and use it whenever it > rejoined the group after a failure.  Since the ID is random and provided by the server, it can't be spoofed or accidentally reused by a misconfigured client.

It is actually the current consumer behavior, and we are discussing whether to enforce member id when accepting join group request in this JIRA<https://issues.apache.org/jira/browse/KAFKA-7610>. Note that we don't have way to persist member ids generated by broker on client side through restarts. Each time restarted consumers will use "unknown member id" to join again. This approach has been discussed<https://github.com/apache/kafka/pull/5176> before and we believe it is fragile to remember ids on client side. That's why we propose KIP-345 on top of it.

Thanks a lot!

Boyang
________________________________
From: Colin McCabe <cm...@apache.org>
Sent: Saturday, November 17, 2018 2:39 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

As Jason said, there are definitely scenarios where we know how many group members we expect ahead of time.  It would be nice if we could distinguish between the error case of "we expected 5 clients in the group, but one failed" and a case like "4 clients started up quickly but the 5th took an extra 2 seconds."  We can sandbag the group rebalance delay, but that's a hack which has clear disadvantages.

It would also be nice to be able to detect when a group member left the group briefly but then came back.

I think both of these issues could be solved by having some broker-side metadata about groups which is configured through the admin client.  If there was an "expected group size," stored on the broker-side, then we could rebalance immediately whenever the group size reached that size.  Otherwise, we could apply the rebalance delay, like now.  This would give lower latency when setting things up.

Expected group size is just an expectation, so the group would be allowed to get bigger than that.  We could also have another number which was the maximum group size.  This really would be a hard upper limit on the size of the group, which admins could optionally configure.

When a new client joined a group, the server could send back a unique random 64-bit member ID.  The client could hold on to this ID and use it whenever it rejoined the group after a failure.  Since the ID is random and provided by the server, it can't be spoofed or accidentally reused by a misconfigured client.

best,
Colin

On Fri, Nov 16, 2018, at 00:04, Jason Gustafson wrote:
> >
> > If we initialize a set of member names (I assume ids = names here) on
> > broker through Admin API, the client needs to pick up this information
> > simultaneously which I doubt if there is a generic way to achieve that? It
> > would also make the scaling operations difficult if we need to define the
> > member names every time we change the member set which is an extra
> > operation burden. From my daily ops experience, dynamically generate member
> > names on client side would be easier. Is there a good approach to address
> > this issue?
>
>
> Yeah, that's a good question. I'm hoping someone with more kubernetes
> experience will jump in here. Basically my goal is to have an approach
> which maps nicely to StatefulSets (
> https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fkubernetes.io%2Fdocs%2Ftutorials%2Fstateful-application%2Fbasic-stateful-set%2F&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=xalo%2BUoajCHoUg1cQ8tgR%2FI5yjiT6d%2FUnqcNbWi7M4s%3D&amp;reserved=0).
> The pods in a stateful set have an ordinal index, which sounds similar to
> the static ids that I was describing. You can scale up and down a stateful
> set, but you would need a plugin to grow and shrink the consumer group.
> Sounds like it could work, but I'm not sure if it's the best way.
>
> At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> > critical streaming services abnormality. One of the burden was the night
> > shift which requires the oncaller to quickly resolve the issue and get the
> > streaming application back on track, however there is a chance of miss. My
> > concern was that if we forfeit the timeout on static membership to trigger
> > rebalance, missing some pages during midnight could be negatively
> > impacting the system performance since we may realize that some partitions
> > stop working for a couple of hours already until next morning. So
> > registration timeout serves as the "last line of defense" to guarantee
> > liveness if no human intervention jumps in.
>
>
> Thanks, this is helpful background. I agree this is a risk in the approach
> I've suggested. If we take a step back, I think there are two gaps in the
> protocol for stateful applications:
>
> 1. We don't have a way to detect the same member across failures or
> restarts. I think streams has some heuristic to try and handle the common
> cases (such as rolling restarts), but the proposal here solves the problem
> in a more robust way.
>
> 2. We don't have a way to know what the expected membership of the group
> is. This leads us to try tricks like inserting delays into the rebalance
> logic so that the group membership has time to stabilize before we make any
> decisions. In your proposal, we have an expansion timeout, which is
> basically the same thing as far as I can tell.
>
> I think the first problem is the most important, but it would be nice if we
> can solve the second problem as well. If we have a way to indicate the
> expected group members, then the group can respond to a change much more
> quickly. There would be no need to wait 5 minutes for all members to join
> and it would be robust in the presence of failures. Ironically, static
> membership in this case makes the group more dynamic ;).
>
> That said, I can see how the registration timeout would be an attractive
> safety net in some cases. Perhaps it would be good enough if we have a way
> to pre-register group members administratively? Members can still be
> expired due to inactivity and we would have a way to get around the
> rebalance delays. Would that work?
>
> Thanks,
> Jason
>
>
> On Wed, Nov 14, 2018 at 10:24 PM, Boyang Chen <bc...@outlook.com> wrote:
>
> > Thank you for the clarification Jason! The proposals make sense here and
> > let me continue the discussion.
> >
> > > Then the ids would be determined using some convention. Most likely, we
> > would just use sequential numbers 0, 1, 2,
> > > etc. We do the same thing for partition ids.
> >
> >
> > If we initialize a set of member names (I assume ids = names here) on
> > broker through Admin API, the client needs to pick up this information
> > simultaneously which I doubt if there is a generic way to achieve that? It
> > would also make the scaling operations difficult if we need to define the
> > member names every time we change the member set which is an extra
> > operation burden. From my daily ops experience, dynamically generate member
> > names on client side would be easier. Is there a good approach to address
> > this issue?
> >
> > > I was thinking that the registration is specified ahead of time and
> > remains valid until changed. It would be more like a
> > > replica assignment. We don't move partitions just because a broker is
> > down.
> > > The expectation is that it the broker will eventually return.
> >
> >
> > At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> > critical streaming services abnormality. One of the burden was the night
> > shift which requires the oncaller to quickly resolve the issue and get the
> > streaming application back on track, however there is a chance of miss. My
> > concern was that if we forfeit the timeout on static membership to trigger
> > rebalance, missing some pages during midnight could be negatively impacting
> > the system performance since we may realize that some partitions stop
> > working for a couple of hours already until next morning. So registration
> > timeout serves as the "last line of defense" to guarantee liveness if no
> > human intervention jumps in.
> >
> >
> > I'm very interested in the replication protocol currently implemented on
> > Kafka, but I'm not familiar with it. If we do have mechanism to handle
> > issues like I mentioned above for replication (auto healing during
> > mid-night if one broker is never back), we could continue discussing the
> > new approaches to have basic guarantee of consumer group liveness.
> >
> >
> > The discussion so far is to make sure that all the design approaches we
> > have taken are pointing to real scenarios. Once we clarify the scenarios,
> > we would definitely propose better solution on top of it. I hope these
> > discussions make sense. Thanks again for helping make the design solid!
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Jason Gustafson <ja...@confluent.io>
> > Sent: Thursday, November 15, 2018 9:54 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > >
> > > I feel this would make the current protocol harder to use. For example,
> > on
> > > KStream we typically would expect (number of instances * number of
> > threads
> > > per instance) consumers. Giving out all this many member names in a list
> > > may not be easy, compared with dynamic generation of member names, at
> > least
> > > for KStream use case.
> >
> >
> > That's a fair point. What I had in mind is for the API to specify the
> > number of consumers in the group. Then the ids would be determined using
> > some convention. Most likely, we would just use sequential numbers 0, 1, 2,
> > etc. We do the same thing for partition ids. Streams folks can chime in and
> > say whether that would work or not.
> >
> > So this suggests we will not rely on heartbeat and commit interval to
> > > trigger rebalance? Even in static membership, I feel tracking active
> > > members is still required to be handled by broker since not all users are
> > > fully equipped with monitoring tools, otherwise users will feel
> > > uncomfortable using static membership solely for reducing rebalance
> > purpose.
> >
> >
> > Let me clarify. I think it is still useful to distinguish the liveness of
> > the consumer using the heartbeat mechanism and to propagate that
> > information to the leader during rebalances. This gives the group leader
> > the option (but not the requirement) to change the partition assignment for
> > inactive members. My suggestion was more about how long the static
> > registration would remain valid. In the current proposal we have a
> > registration timeout, and I think Mayuresh is fair to point out the
> > potential confusion with the session timeout. It's kind of the same, but
> > not exactly. So instead, I was thinking that the registration is specified
> > ahead of time and remains valid until changed. It would be more like a
> > replica assignment. We don't move partitions just because a broker is down.
> > The expectation is that it the broker will eventually return. Similarly for
> > stateful applications, we would have the option to do the same thing. The
> > registration is fixed until someone changes it.
> >
> > Does that make sense?
> >
> > Thanks,
> > Jason
> >
> >
> > On Wed, Nov 14, 2018 at 2:46 PM, Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Jason for the suggestions! I update the KIP with full schema
> > > changes.
> > >
> > > > we offer an admin API that lets a user define the expected members of
> > > the group.
> > >
> > > I feel this would make the current protocol harder to use. For example,
> > on
> > > KStream we typically would expect (number of instances * number of
> > threads
> > > per instance) consumers. Giving out all this many member names in a list
> > > may not be easy, compared with dynamic generation of member names, at
> > least
> > > for KStream use case.
> > >
> > > > For the sake of discussion, I was wondering if we could just say that
> > > static members do not expire.
> > >
> > >
> > > So this suggests we will not rely on heartbeat and commit interval to
> > > trigger rebalance? Even in static membership, I feel tracking active
> > > members is still required to be handled by broker since not all users are
> > > fully equipped with monitoring tools, otherwise users will feel
> > > uncomfortable using static membership solely for reducing rebalance
> > purpose.
> > >
> > >
> > > For the admin API design, I'm simplifying the join group request
> > handling,
> > > while using admin tool to switch between static and dynamic membership
> > and
> > > set the two corresponding timeouts. Do you think this approach makes
> > sense?
> > > The version one implementation will be much more clean if we handle
> > > membership change through user intervention.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Jason Gustafson <ja...@confluent.io>
> > > Sent: Wednesday, November 14, 2018 9:31 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hey Boyang,
> > >
> > > Thanks for the updates. From a high level, I think this actually
> > > complements Konstantine's writeup on incremental rebalancing. The gap
> > we're
> > > addressing is providing a way to bind the the partition assignment of a
> > > group to a set of user-provided ids so that we are not so reliant on the
> > > group's immediate state. For example, these ids might identify the state
> > > store volume for particular streams instances. This is basically what you
> > > need to work well with k8s stateful sets (as far as I understand them).
> > >
> > > One key decision is how we would define and update the expected static
> > > members in a consumer group. The mechanics of the registration and
> > > expansion timeouts feel a little bit clunky. For the sake of discussion,
> > I
> > > was wondering if we could just say that static members do not expire.
> > > Instead, we offer an admin API that lets a user define the expected
> > members
> > > of the group. This API could be used to both grow and shrink a group.
> > This
> > > would solve the rebalancing problems when applications are initially
> > > bootstrapped or when they are restarted because we would always know how
> > > many members should be in a group. What do you think?
> > >
> > > By the way, it would be helpful to include the full schema definition for
> > > any protocol changes in the proposal.
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > > On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com>
> > wrote:
> > >
> > > > Thanks Mayuresh for the feedback! Do you have a quick example for
> > passing
> > > > in consumer config dynamically? I mainly use Kafka Streams at my daily
> > > work
> > > > so probably missing the idea how to do it in the current consumer
> > > setting.
> > > >
> > > >
> > > > For differentiating session timeout and registration timeout, I would
> > try
> > > > to enhance the documentation in the first stage to see how people react
> > > to
> > > > the confusion (would be great if they feel straightforward!). Since one
> > > > doesn't have to fully understand the difference unless defining the new
> > > > config "member name", for current users we could buy some time to
> > listen
> > > to
> > > > their understandings and improve our documentation correspondingly in
> > the
> > > > follow-up KIPs.
> > > >
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Mayuresh Gharat <gh...@gmail.com>
> > > > Sent: Sunday, November 11, 2018 1:06 PM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > > specifying member id
> > > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the reply.
> > > >
> > > > Please find the replies inline below :
> > > > For having a consumer config at runtime, I think it's not necessary to
> > > > address in this KIP because most companies run sidecar jobs through
> > > daemon
> > > > software like puppet. It should be easy to change the config through
> > > script
> > > > or UI without actual code change. We still want to leave flexibility
> > for
> > > > user to define member name as they like.
> > > > ---- This might be little different for companies that use
> > configuration
> > > > management tools that does not allow the applications to define/change
> > > the
> > > > configs dynamically. For example, if we use something similar to spring
> > > to
> > > > pull in the configs for the KafkaConsumer and pass it to the
> > constructor
> > > to
> > > > create the KafkaConsumer object, it will be hard to specify a unique
> > > value
> > > > to the "MEMBER_NAME" config unless someone deploying the app generates
> > a
> > > > unique string for this config outside the deployment workflow and
> > copies
> > > it
> > > > statically before starting up each consumer instance. Unless we can
> > > loosen
> > > > the criteria for uniqueness of this config value, for each consumer
> > > > instance in the consumer group, I am not sure of a better way of
> > > > addressing this. If we don't want to loosen the criteria, then
> > providing
> > > a
> > > > dynamic way to pass this in at runtime, would put the onus of having
> > the
> > > > same unique value each time a consumer is restarted, on to the
> > > application
> > > > that is running the consumer.
> > > >
> > > > I just updated the kip about having both "registration timeout" and
> > > > "session timeout". The benefit of having two configs instead of one is
> > to
> > > > reduce the mental burden for operation, for example user just needs to
> > > > unset "member name" to cast back to dynamic membership without worrying
> > > > about tuning the "session timeout" back to a smaller value.
> > > > --- That is a good point. I was thinking, if both the configs are
> > > > specified, it would be confusing for the end user without understanding
> > > the
> > > > internals of the consumer and its interaction with group coordinator,
> > as
> > > > which takes precedence when and how it affects the consumer behavior.
> > > Just
> > > > my 2 cents.
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > > > Hey Mayuresh,
> > > > >
> > > > >
> > > > > thanks for the thoughtful questions! Let me try to answer your
> > > questions
> > > > > one by one.
> > > > >
> > > > >
> > > > > For having a consumer config at runtime, I think it's not necessary
> > to
> > > > > address in this KIP because most companies run sidecar jobs through
> > > > daemon
> > > > > software like puppet. It should be easy to change the config through
> > > > script
> > > > > or UI without actual code change. We still want to leave flexibility
> > > for
> > > > > user to define member name as they like.
> > > > >
> > > > >
> > > > > I just updated the kip about having both "registration timeout" and
> > > > > "session timeout". The benefit of having two configs instead of one
> > is
> > > to
> > > > > reduce the mental burden for operation, for example user just needs
> > to
> > > > > unset "member name" to cast back to dynamic membership without
> > worrying
> > > > > about tuning the "session timeout" back to a smaller value.
> > > > >
> > > > >
> > > > > For backup topic, I think it's a low-level detail which could be
> > > > addressed
> > > > > in the implementation. I feel no preference of adding a new topic vs
> > > > reuse
> > > > > consumer offsets topic. I will do more analysis and make a trade-off
> > > > > comparison. Nice catch!
> > > > >
> > > > >
> > > > > I hope the explanations make sense to you. I will keep polishing on
> > the
> > > > > edge cases and details.
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Mayuresh Gharat <gh...@gmail.com>
> > > > > Sent: Saturday, November 10, 2018 10:25 AM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP and sorry for being late to the party. This KIP is
> > > > > really useful for us at Linkedin.
> > > > >
> > > > > I had a few questions :
> > > > >
> > > > > The idea of having static member name seems nice, but instead of a
> > > > config,
> > > > > would it be possible for it to be passed in to the consumer at
> > runtime?
> > > > > This is because an app might want to decide the config value at
> > runtime
> > > > > using its host information for example, to generate the unique member
> > > > name.
> > > > >
> > > > > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > > > > wondering if we can reuse the session timeout here. This might help
> > us
> > > to
> > > > > have one less config on the consumer.
> > > > >
> > > > > The KIP also talks about adding another internal topic
> > > > "static_member_map".
> > > > > Would the semantics (GroupCoordinator broker, topic configs) be the
> > > same
> > > > as
> > > > > __consumer_offsets topic?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > >
> > > > > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > > >
> > > > > > I took a quick pass of the proposal. First I would say it's a very
> > > > > > brilliant initiative from Konstantine and Confluent folks. To draft
> > > up
> > > > a
> > > > > > proposal like this needs deep understanding of the rebalance
> > > protocol!
> > > > I
> > > > > > summarized some thoughts here.
> > > > > >
> > > > > >
> > > > > > Overall the motivations of the two proposals align on that:
> > > > > >
> > > > > >   1.  Both believe the invariant resource (belonging to the same
> > > > process)
> > > > > > should be preserved across rebalance.
> > > > > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > > > > redistribution. I don't use rebalance here since part one of the
> > > > > > cooperative proposal could potentially introduce more rebalances
> > but
> > > > only
> > > > > > on must-move resources.
> > > > > >   3.  Scale up/down and rolling bounce are causing unnecessary
> > > resource
> > > > > > shuffling that need to be mitigated.
> > > > > >
> > > > > >
> > > > > > On motivation level, I think both approach could solve/mitigate the
> > > > above
> > > > > > issues. They are just different in design philosophy, or I would
> > say
> > > > the
> > > > > > perspective difference between framework user and algorithm
> > designer.
> > > > > >
> > > > > >
> > > > > > Two proposals have different focuses. KIP-345 is trying to place
> > more
> > > > > > fine-grained control on the broker side to reduce the unnecessary
> > > > > > rebalances, while keeping the client logic intact. This is pretty
> > > > > intuitive
> > > > > > cause-effect for normal developers who are not very familiar with
> > > > > rebalance
> > > > > > protocol. As a developer working with Kafka Streams daily, I'd be
> > > happy
> > > > > to
> > > > > > see a simplified rebalance protocol and just focus on maintaining
> > the
> > > > > > stream/consumer jobs. Too many rebalances raised my concern on the
> > > job
> > > > > > health. To be concise, static membership has the advantage of
> > > reducing
> > > > > > mental burden.
> > > > > >
> > > > > >
> > > > > > Cooperative proposal takes thoughtful approach on client side. We
> > > want
> > > > to
> > > > > > have fine-grained control on the join/exit group behaviors and make
> > > the
> > > > > > current dynamic membership better to address above issues. I do
> > feel
> > > > our
> > > > > > idea crossed on the delayed rebalance when we scale up/down, which
> > > > could
> > > > > > potentially reduce the state shuffling and decouple the behavior
> > from
> > > > > > session timeout which is already overloaded.  In this sense, I
> > > believe
> > > > > both
> > > > > > approaches would serve well in making "reasonable rebalance" happen
> > > at
> > > > > the
> > > > > > "right timing".
> > > > > >
> > > > > >
> > > > > > However, based on my understanding, either 345 or cooperative
> > > > rebalancing
> > > > > > is not solving the problem Mike has proposed: could we do a better
> > > job
> > > > at
> > > > > > scaling up/down in ideal timing? My initial response was to
> > introduce
> > > > an
> > > > > > admin API which now I feel is sub-optimal, in that the goal of
> > smooth
> > > > > > transition is to make sure the newly up hosts are actually "ready".
> > > For
> > > > > > example:
> > > > > >
> > > > > >
> > > > > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At
> > > some
> > > > > > time we would like to scale up to 8 hosts, with the current
> > > > improvements
> > > > > we
> > > > > > could reduce 4 potential rebalances to a single one. But the new
> > > hosts
> > > > > are
> > > > > > yet unknown to be "ready" if they need to reconstruct the local
> > > state.
> > > > To
> > > > > > be actually ready, we need 4 standby tasks running on those empty
> > > hosts
> > > > > and
> > > > > > leader needs to wait for the signal of "replay/reconstruct
> > complete"
> > > to
> > > > > > actually involve them into the main consumer group. Otherwise,
> > > > rebalance
> > > > > > just kills our performance since we need to wait indefinite long
> > for
> > > > task
> > > > > > migration.
> > > > > >
> > > > > >
> > > > > > The scale down is also tricky such that we are not able to define a
> > > > > "true"
> > > > > > leave of a member. Rebalance immediately after "true" leaves are
> > most
> > > > > > optimal comparing with human intervention. Does this make sense?
> > > > > >
> > > > > >
> > > > > > My intuition is that cooperative approach which was implemented on
> > > the
> > > > > > client side could better handle scaling cases than KIP 345, since
> > it
> > > > > > involves a lot of algorithmic changes to define "replaying" stage,
> > > > which
> > > > > I
> > > > > > feel would over-complicate broker logic if implemented on
> > > coordinator.
> > > > If
> > > > > > we let 345 focus on reducing unnecessary rebalance, and let
> > > cooperative
> > > > > > approach focus on judging best timing of scale up/down, the two
> > > efforts
> > > > > > could be aligned. In long term, I feel the more complex improvement
> > > of
> > > > > > consumer protocol should happen on client side instead of server
> > side
> > > > > which
> > > > > > is easier to test and has less global impact for the entire Kafka
> > > > > > production cluster.
> > > > > >
> > > > > >
> > > > > > Thanks again to Konstantine, Matthias and other folks in coming up
> > > with
> > > > > > this great client proposal. This is great complementation to KIP
> > 345.
> > > > In
> > > > > a
> > > > > > high level, we are not having any collision on the path and both
> > > > > proposals
> > > > > > are making sense here. Just need better sync to avoid duplicate
> > > effort
> > > > :)
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > >
> > > > > > ________________________________
> > > > > > From: Boyang Chen <bc...@outlook.com>
> > > > > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > > > > To: dev@kafka.apache.org
> > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > > by
> > > > > > specifying member id
> > > > > >
> > > > > > Thanks Matthias for bringing this awesome proposal up! I shall
> > take a
> > > > > > deeper look and make a comparison between the two proposals.
> > > > > >
> > > > > >
> > > > > > Meanwhile for the scale down specifically for stateful streaming,
> > we
> > > > > could
> > > > > > actually introduce a new status called "learner" where the newly up
> > > > hosts
> > > > > > could try to catch up with the assigned task progress first before
> > > > > > triggering the rebalance, from which we don't see a sudden dip on
> > the
> > > > > > progress. However, it is built on top of the success of KIP-345.
> > > > > >
> > > > > >
> > > > > > ________________________________
> > > > > > From: Matthias J. Sax <ma...@confluent.io>
> > > > > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > > > > To: dev@kafka.apache.org
> > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > > by
> > > > > > specifying member id
> > > > > >
> > > > > > Hey,
> > > > > >
> > > > > > there was quite a pause on this KIP discussion and in the mean
> > time,
> > > a
> > > > > > new design for incremental cooporative rebalance was suggested:
> > > > > >
> > > > > >
> > > > > >
> > > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FIncrementa&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=D7DhmFWNQPCiZPqE%2FEZff6HEu9wxbdAKAdklhCJ0rUY%3D&amp;reserved=0
> > > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FIncrem&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=6Q0ppNImKfhTvT9Y2PiKOCASpv4Mp4dBtOdCGaJcvJM%3D&amp;reserved=0
> > > > ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> > > > cwiki.apache.org
> > > > Rebalancing between distributed application processes in Apache Kafka
> > was
> > > > enhanced considerably when it was decoupled as logic from Kafka brokers
> > > and
> > > > was moved as responsibility to the clients and specifically to Kafka
> > > > Consumer. This pattern has been working robustly for quite a while now
> > > and
> > > > has ...
> > > >
> > > >
> > > >
> > > > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FIncrementa&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=D7DhmFWNQPCiZPqE%2FEZff6HEu9wxbdAKAdklhCJ0rUY%3D&amp;reserved=0
> > > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > > > >
> > > > > cwiki.apache.org
> > > > > Rebalancing between distributed application processes in Apache Kafka
> > > was
> > > > > enhanced considerably when it was decoupled as logic from Kafka
> > brokers
> > > > and
> > > > > was moved as responsibility to the clients and specifically to Kafka
> > > > > Consumer. This pattern has been working robustly for quite a while
> > now
> > > > and
> > > > > has ...
> > > > >
> > > > >
> > > > >
> > > > > >
> > > > > >
> > > > > > We should make sure that the proposal and this KIP align to each
> > > other.
> > > > > > Thoughts?
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > > > > Hey Mike,
> > > > > > >
> > > > > > >
> > > > > > > thanks for the feedback, the two question are very thoughtful!
> > > > > > >
> > > > > > >
> > > > > > >> 1) I am a little confused about the distinction for the leader.
> > If
> > > > the
> > > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > > quickly
> > > > > > comes up) to update application code, will a rebalance be
> > triggered?
> > > I
> > > > >
> > > > > do
> > > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > > >
> > > > > > > For Q1 my intention was to minimize the change within one KIP,
> > > since
> > > > > the
> > > > > > leader rejoining case could be addressed separately.
> > > > > > >
> > > > > > >
> > > > > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > > gracefully increase the number of nodes in the cluster. I think we
> > > need
> > > > > to
> > > > > > support graceful shrink down as well. If I set the registration
> > > timeout
> > > > > to
> > > > > > 5 minutes > to handle rolling restarts or intermittent failures
> > > without
> > > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > > group
> > > > to
> > > > > > rebalance if I am intentionally removing a node from the cluster. I
> > > am
> > > > > not
> > > > > > sure the best way to > do this. One idea I had was adding the
> > ability
> > > > > for a
> > > > > > CLI or Admin API to force a rebalance of the group. This would
> > allow
> > > > for
> > > > > an
> > > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > > registration timeout on > shrink down. What do you think?
> > > > > > >
> > > > > > > For 2) my understanding is that for scaling down case it is
> > better
> > > to
> > > > > be
> > > > > > addressed by CLI tool than code logic, since only by human
> > evaluation
> > > > we
> > > > > > could decide whether it is a "right timing" -- the time when all
> > the
> > > > > > scaling down consumers are offline -- to kick in rebalance. Unless
> > we
> > > > > > introduce another term on coordinator which indicates the target
> > > > consumer
> > > > > > group size, broker will find it hard to decide when to start
> > > rebalance.
> > > > > So
> > > > > > far I prefer to hold the implementation for that, but agree we
> > could
> > > > > > discuss whether we want to introduce admin API in this KIP or a
> > > > separate
> > > > > > one.
> > > > > > >
> > > > > > >
> > > > > > > Thanks again for the proposed ideas!
> > > > > > >
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > ________________________________
> > > > > > > From: Mike Freyberger <mi...@xandr.com>
> > > > > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > > > > To: dev@kafka.apache.org
> > > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by
> > > > > > specifying member id
> > > > > > >
> > > > > > > Boyang,
> > > > > > >
> > > > > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > > > > >
> > > > > > > 1) I am a little confused about the distinction for the leader.
> > If
> > > > the
> > > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > > quickly
> > > > > > comes up) to update application code, will a rebalance be
> > triggered?
> > > I
> > > > do
> > > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > > >
> > > > > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > > gracefully increase the number of nodes in the cluster. I think we
> > > need
> > > > > to
> > > > > > support graceful shrink down as well. If I set the registration
> > > timeout
> > > > > to
> > > > > > 5 minutes to handle rolling restarts or intermittent failures
> > without
> > > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > > group
> > > > to
> > > > > > rebalance if I am intentionally removing a node from the cluster. I
> > > am
> > > > > not
> > > > > > sure the best way to do this. One idea I had was adding the ability
> > > > for a
> > > > > > CLI or Admin API to force a rebalance of the group. This would
> > allow
> > > > for
> > > > > an
> > > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > > registration timeout on shrink down. What do you think?
> > > > > > >
> > > > > > > Mike
> > > > > > >
> > > > > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com>
> > wrote:
> > > > > > >
> > > > > > >     Btw, I updated KIP 345 based on my understanding. Feel free
> > to
> > > > take
> > > > > > another round of look:
> > > > > > >
> > > > > > >
> > > > > >
> > > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-345%253A&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=xckMR4u1zSI%2F8hq%2FLSdk7AXWc6Uq%2FC6BJP%2FD79sO2As%3D&amp;reserved=0
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > > KIP-345: Introduce static membership protocol to reduce ...<
> > > > > >
> > > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-345%253A&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=xckMR4u1zSI%2F8hq%2FLSdk7AXWc6Uq%2FC6BJP%2FD79sO2As%3D&amp;reserved=0
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > >
> > > > > > > cwiki.apache.org
> > > > > > > For stateful applications, one of the biggest performance
> > > bottleneck
> > > > is
> > > > > > the state shuffling. In Kafka consumer, there is a concept called
> > > > > > "rebalance" which means that for given M partitions and N consumers
> > > in
> > > > > one
> > > > > > consumer group, Kafka will try to balance the load between
> > consumers
> > > > and
> > > > > > ideally have ...
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > > > > >
> > > > > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fdisplay%2FKAFKA%2FKIP-345%253A&amp;data=02%7C01%7C%7Cd71a3252b2d5417ed9ae08d64bf2d738%7C84df9e7fe9f640afb435aaaaaaaaaaaa%7C1%7C0%7C636779903694742722&amp;sdata=xckMR4u1zSI%2F8hq%2FLSdk7AXWc6Uq%2FC6BJP%2FD79sO2As%3D&amp;reserved=0
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > >
> > > > > > >     cwiki.apache.org
> > > > > > >     For stateful applications, one of the biggest performance
> > > > > bottleneck
> > > > > > is the state shuffling. In Kafka consumer, there is a concept
> > called
> > > > > > "rebalance" which means that for given M partitions and N consumers
> > > in
> > > > > one
> > > > > > consumer group, Kafka will try to balance the load between
> > consumers
> > > > and
> > > > > > ideally have ...
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     ________________________________
> > > > > > >     From: Boyang Chen <bc...@outlook.com>
> > > > > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > > > > >     To: dev@kafka.apache.org
> > > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > > by specifying member id
> > > > > > >
> > > > > > >     Thanks everyone for the input on this thread! (Sorry it's
> > been
> > > a
> > > > > > while) I feel that we are very close to the final solution.
> > > > > > >
> > > > > > >
> > > > > > >     Hey Jason and Mike, I have two quick questions on the new
> > > > features
> > > > > > here:
> > > > > > >
> > > > > > >       1.  so our proposal is that until we add a new static
> > member
> > > > into
> > > > > > the group (scale up), we will not trigger rebalance until the
> > > > > "registration
> > > > > > timeout"( the member has been offline for too long)? How about
> > > leader's
> > > > > > rejoin request, I think we should still trigger rebalance when that
> > > > > > happens, since the consumer group may have new topics to consume?
> > > > > > >       2.  I'm not very clear on the scale up scenario in static
> > > > > > membership here. Should we fallback to dynamic membership while
> > > > > > adding/removing hosts (by setting member.name = null), or we still
> > > > want
> > > > > > to add instances with `member.name` so that we eventually
> > > > expand/shrink
> > > > > > the static membership? I personally feel the easier solution is to
> > > spin
> > > > > up
> > > > > > new members and wait until either the same "registration timeout"
> > or
> > > a
> > > > > > "scale up timeout" before starting the rebalance. What do you
> > think?
> > > > > > >
> > > > > > >     Meanwhile I will go ahead to make changes to the KIP with our
> > > > newly
> > > > > > discussed items and details. Really excited to see the design has
> > > > become
> > > > > > more solid.
> > > > > > >
> > > > > > >     Best,
> > > > > > >     Boyang
> > > > > > >
> > > > > > >     ________________________________
> > > > > > >     From: Jason Gustafson <ja...@confluent.io>
> > > > > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > > > > >     To: dev
> > > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > > by specifying member id
> > > > > > >
> > > > > > >     Hey Mike,
> > > > > > >
> > > > > > >     Yeah, that's a good point. A long "registration timeout" may
> > > not
> > > > be
> > > > > > a great
> > > > > > >     idea. Perhaps in practice you'd set it long enough to be able
> > > to
> > > > > > detect a
> > > > > > >     failure and provision a new instance. Maybe on the order of
> > 10
> > > > > > minutes is
> > > > > > >     more reasonable.
> > > > > > >
> > > > > > >     In any case, it's probably a good idea to have an
> > > administrative
> > > > > way
> > > > > > to
> > > > > > >     force deregistration. One option is to extend the
> > DeleteGroups
> > > > API
> > > > > > with a
> > > > > > >     list of members names.
> > > > > > >
> > > > > > >     -Jason
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > > > > mfreyberger@appnexus.com>
> > > > > > >     wrote:
> > > > > > >
> > > > > > >     > Jason,
> > > > > > >     >
> > > > > > >     > Regarding step 4 in your proposal which suggests beginning
> > a
> > > > long
> > > > > > timer
> > > > > > >     > (30 minutes) when a static member leaves the group, would
> > > there
> > > > > > also be the
> > > > > > >     > ability for an admin to force a static membership
> > expiration?
> > > > > > >     >
> > > > > > >     > I'm thinking that during particular types of outages or
> > > > upgrades
> > > > > > users
> > > > > > >     > would want forcefully remove a static member from the
> > group.
> > > > > > >     >
> > > > > > >     > So the user would shut the consumer down normally, which
> > > > wouldn't
> > > > > > trigger
> > > > > > >     > a rebalance. Then the user could use an admin CLI tool to
> > > force
> > > > > > remove that
> > > > > > >     > consumer from the group, so the TopicPartitions that were
> > > > > > previously owned
> > > > > > >     > by that consumer can be released.
> > > > > > >     >
> > > > > > >     > At a high level, we need consumer groups to gracefully
> > handle
> > > > > > intermittent
> > > > > > >     > failures and permanent failures. Currently, the consumer
> > > group
> > > > > > protocol
> > > > > > >     > handles permanent failures well, but does not handle
> > > > intermittent
> > > > > > failures
> > > > > > >     > well (it creates unnecessary rebalances). I want to make
> > sure
> > > > the
> > > > > > overall
> > > > > > >     > solution here handles both intermittent failures and
> > > permanent
> > > > > > failures,
> > > > > > >     > rather than sacrificing support for permanent failures in
> > > order
> > > > > to
> > > > > > provide
> > > > > > >     > support for intermittent failures.
> > > > > > >     >
> > > > > > >     > Mike
> > > > > > >     >
> > > > > > >     > Sent from my iPhone
> > > > > > >     >
> > > > > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > wrote:
> > > > > > >     > >
> > > > > > >     > > Hey Guozhang,
> > > > > > >     > >
> > > > > > >     > > Responses below:
> > > > > > >     > >
> > > > > > >     > > Originally I was trying to kill more birds with one stone
> > > > with
> > > > > > KIP-345,
> > > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > > shutting
> > > > > > down a
> > > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> > early
> > > > > > email), and
> > > > > > >     > >> hence proposing to have a pure static-membership
> > protocol.
> > > > But
> > > > > > thinking
> > > > > > >     > >> twice about it I now feel it may be too ambitious and
> > > worth
> > > > > > fixing in
> > > > > > >     > >> another KIP.
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > I was considering an extension to support
> > > pre-initialization
> > > > of
> > > > > > the
> > > > > > >     > static
> > > > > > >     > > members of the group, but I agree we should probably
> > leave
> > > > this
> > > > > > problem
> > > > > > >     > for
> > > > > > >     > > future work.
> > > > > > >     > >
> > > > > > >     > > 1. How this longish static member expiration timeout
> > > defined?
> > > > > Is
> > > > > > it via a
> > > > > > >     > >> broker, hence global config, or via a client config
> > which
> > > > can
> > > > > be
> > > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > I am not too sure. I tend to lean toward server-side
> > > configs
> > > > > > because they
> > > > > > >     > > are easier to evolve. If we have to add something to the
> > > > > > protocol, then
> > > > > > >     > > we'll be stuck with it forever.
> > > > > > >     > >
> > > > > > >     > > 2. Assuming that for static members, LEAVE_GROUP request
> > > will
> > > > > not
> > > > > > >     > trigger a
> > > > > > >     > >> rebalance immediately either, similar to session
> > timeout,
> > > > but
> > > > > > only the
> > > > > > >     > >> longer member expiration timeout, can we remove the
> > > > internal "
> > > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > > walk-around
> > > > > > >     > then?
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > Yeah, I hope we can ultimately get rid of it, but we may
> > > need
> > > > > it
> > > > > > for
> > > > > > >     > > compatibility with older brokers. A related question is
> > > what
> > > > > > should be
> > > > > > >     > the
> > > > > > >     > > behavior of the consumer if `member.name` is provided
> > but
> > > > the
> > > > > > broker
> > > > > > >     > does
> > > > > > >     > > not support it? We could either fail or silently
> > downgrade
> > > to
> > > > > > dynamic
> > > > > > >     > > membership.
> > > > > > >     > >
> > > > > > >     > > -Jason
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > >     > wrote:
> > > > > > >     > >>
> > > > > > >     > >> Hey Jason,
> > > > > > >     > >>
> > > > > > >     > >> I like your idea to simplify the upgrade protocol to
> > allow
> > > > > > co-exist of
> > > > > > >     > >> static and dynamic members. Admittedly it may make the
> > > > > > coordinator-side
> > > > > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > > > > >     > >>
> > > > > > >     > >> Originally I was trying to kill more birds with one
> > stone
> > > > with
> > > > > > KIP-345,
> > > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > > shutting
> > > > > > down a
> > > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> > early
> > > > > > email), and
> > > > > > >     > >> hence proposing to have a pure static-membership
> > protocol.
> > > > But
> > > > > > thinking
> > > > > > >     > >> twice about it I now feel it may be too ambitious and
> > > worth
> > > > > > fixing in
> > > > > > >     > >> another KIP. With that, I think what you've proposed
> > here
> > > > is a
> > > > > > good way
> > > > > > >     > to
> > > > > > >     > >> go for KIP-345 itself.
> > > > > > >     > >>
> > > > > > >     > >> Note there are a few details in your proposal we'd still
> > > > need
> > > > > > to figure
> > > > > > >     > >> out:
> > > > > > >     > >>
> > > > > > >     > >> 1. How this longish static member expiration timeout
> > > > defined?
> > > > > > Is it via
> > > > > > >     > a
> > > > > > >     > >> broker, hence global config, or via a client config
> > which
> > > > can
> > > > > be
> > > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > > >     > >>
> > > > > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> > > > will
> > > > > > not
> > > > > > >     > trigger a
> > > > > > >     > >> rebalance immediately either, similar to session
> > timeout,
> > > > but
> > > > > > only the
> > > > > > >     > >> longer member expiration timeout, can we remove the
> > > > internal "
> > > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > > walk-around
> > > > > > >     > then?
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> Guozhang
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > > > > jason@confluent.io>
> > > > > > >     > >> wrote:
> > > > > > >     > >>
> > > > > > >     > >>> Hey All,
> > > > > > >     > >>>
> > > > > > >     > >>> Nice to see some solid progress on this. It sounds like
> > > one
> > > > > of
> > > > > > the
> > > > > > >     > >>> complications is allowing static and dynamic
> > registration
> > > > to
> > > > > > coexist.
> > > > > > >     > I'm
> > > > > > >     > >>> wondering if we can do something like the following:
> > > > > > >     > >>>
> > > > > > >     > >>> 1. Statically registered members (those joining the
> > group
> > > > > with
> > > > > > a
> > > > > > >     > >> non-null `
> > > > > > >     > >>> member.name`) maintain a session with the coordinator
> > > just
> > > > > > like
> > > > > > >     > dynamic
> > > > > > >     > >>> members.
> > > > > > >     > >>> 2. If a session is active for a static member when a
> > > > > rebalance
> > > > > > begins,
> > > > > > >     > >> then
> > > > > > >     > >>> basically we'll keep the current behavior. The
> > rebalance
> > > > will
> > > > > > await the
> > > > > > >     > >>> static member joining the group.
> > > > > > >     > >>> 3. If a static member does not have an active session,
> > > then
> > > > > the
> > > > > > >     > >> coordinator
> > > > > > >     > >>> will not wait for it to join, but will still include it
> > > in
> > > > > the
> > > > > > >     > rebalance.
> > > > > > >     > >>> The coordinator will forward the cached subscription
> > > > > > information to the
> > > > > > >     > >>> leader and will cache the assignment after the
> > rebalance
> > > > > > completes.
> > > > > > >     > (Note
> > > > > > >     > >>> that we still have the generationId to fence offset
> > > commits
> > > > > > from a
> > > > > > >     > static
> > > > > > >     > >>> zombie if the assignment changes.)
> > > > > > >     > >>> 4. When a static member leaves the group or has its
> > > session
> > > > > > expire, no
> > > > > > >     > >>> rebalance is triggered. Instead, we can begin a timer
> > to
> > > > > > expire the
> > > > > > >     > >> static
> > > > > > >     > >>> registration. This would be a longish timeout (like 30
> > > > > minutes
> > > > > > say).
> > > > > > >     > >>>
> > > > > > >     > >>> So basically static members participate in all
> > rebalances
> > > > > > regardless
> > > > > > >     > >>> whether they have an active session. In a given
> > > rebalance,
> > > > > > some of the
> > > > > > >     > >>> members may be static and some dynamic. The group
> > leader
> > > > can
> > > > > > >     > >> differentiate
> > > > > > >     > >>> the two based on the presence of the `member.name` (we
> > > > have
> > > > > > to add
> > > > > > >     > this
> > > > > > >     > >> to
> > > > > > >     > >>> the JoinGroupResponse). Generally speaking, we would
> > > choose
> > > > > > leaders
> > > > > > >     > >>> preferentially from the active members that support the
> > > > > latest
> > > > > > >     > JoinGroup
> > > > > > >     > >>> protocol and are using static membership. If we have to
> > > > > choose
> > > > > > a leader
> > > > > > >     > >>> with an old version, however, it would see all members
> > in
> > > > the
> > > > > > group
> > > > > > >     > >> (static
> > > > > > >     > >>> or dynamic) as dynamic members and perform the
> > assignment
> > > > as
> > > > > > usual.
> > > > > > >     > >>>
> > > > > > >     > >>> Would that work?
> > > > > > >     > >>>
> > > > > > >     > >>> -Jason
> > > > > > >     > >>>
> > > > > > >     > >>>
> > > > > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > >     > >> wrote:
> > > > > > >     > >>>
> > > > > > >     > >>>> Hello Boyang,
> > > > > > >     > >>>>
> > > > > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > > > > >     > >>>>
> > > > > > >     > >>>> 1. Where will "change-group-timeout" be communicated
> > to
> > > > the
> > > > > > broker?
> > > > > > >     > >> Will
> > > > > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> > > > going
> > > > > > to
> > > > > > >     > >>> piggy-back
> > > > > > >     > >>>> on the existing session-timeout field (assuming that
> > the
> > > > > > original
> > > > > > >     > value
> > > > > > >     > >>>> will not be used anywhere in the static membership any
> > > > > more)?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > > > > timeout to
> > > > > > >     > >> return,
> > > > > > >     > >>>> we shall still trigger rebalance but it could still
> > try
> > > to
> > > > > > catch
> > > > > > >     > >>>> `change-group-timeout`.": what does this mean? I
> > thought
> > > > > your
> > > > > > proposal
> > > > > > >     > >> is
> > > > > > >     > >>>> that for static memberships, the broker will NOT
> > trigger
> > > > > > rebalance
> > > > > > >     > even
> > > > > > >     > >>>> after session-timeout has been detected, but only that
> > > > after
> > > > > > >     > >>>> change-group-timeout
> > > > > > >     > >>>> which is supposed to be longer than session-timeout to
> > > be
> > > > > > defined?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 3. "A join group request with member.name set will be
> > > > > > treated as
> > > > > > >     > >>>> `static-membership` strategy", in this case, how would
> > > the
> > > > > > switch from
> > > > > > >     > >>>> dynamic to static happen, since whoever changed the
> > > > > > member.name to
> > > > > > >     > >>>> not-null
> > > > > > >     > >>>> will be rejected, right?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 4. "just erase the cached mapping, and wait for
> > session
> > > > > > timeout to
> > > > > > >     > >>> trigger
> > > > > > >     > >>>> rebalance should be sufficient." this is also a bit
> > > > unclear
> > > > > > to me: who
> > > > > > >     > >>> will
> > > > > > >     > >>>> erase the cached mapping? Since it is on the
> > > broker-side I
> > > > > > assume that
> > > > > > >     > >>>> broker has to do it. Are you suggesting to use a new
> > > > request
> > > > > > for it?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 5. "Halfway switch": following 3) above, if your
> > > proposal
> > > > is
> > > > > > basically
> > > > > > >     > >> to
> > > > > > >     > >>>> let "first join-request wins", and the strategy will
> > > stay
> > > > as
> > > > > > is until
> > > > > > >     > >> all
> > > > > > >     > >>>> members are gone, then this will also not happen since
> > > > > > whoever used
> > > > > > >     > >>>> different strategy as the first guy who sends
> > join-group
> > > > > > request will
> > > > > > >     > >> be
> > > > > > >     > >>>> rejected right?
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> Guozhang
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > > > > john@confluent.io>
> > > > > > >     > >> wrote:
> > > > > > >     > >>>>
> > > > > > >     > >>>>> This sounds good to me!
> > > > > > >     > >>>>>
> > > > > > >     > >>>>> Thanks for the time you've spent on it,
> > > > > > >     > >>>>> -John
> > > > > > >     > >>>>>
> > > > > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > > > > bchen11@outlook.com>
> > > > > > >     > >>>> wrote:
> > > > > > >     > >>>>>
> > > > > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy
> > > recently
> > > > > and
> > > > > > >     > >> haven't
> > > > > > >     > >>>> got
> > > > > > >     > >>>>>> time to update this thread. To summarize what we
> > come
> > > up
> > > > > so
> > > > > > far,
> > > > > > >     > >> here
> > > > > > >     > >>>> is
> > > > > > >     > >>>>> a
> > > > > > >     > >>>>>> draft updated plan:
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Introduce a new config called `member.name` which
> > is
> > > > > > supposed to
> > > > > > >     > >> be
> > > > > > >     > >>>>>> provided uniquely by the consumer client. The broker
> > > > will
> > > > > > maintain
> > > > > > >     > >> a
> > > > > > >     > >>>>> cache
> > > > > > >     > >>>>>> with [key:member.name, value:member.id]. A join
> > group
> > > > > > request with
> > > > > > >     > >>>>>> member.name set will be treated as
> > > `static-membership`
> > > > > > strategy,
> > > > > > >     > >> and
> > > > > > >     > >>>>> will
> > > > > > >     > >>>>>> reject any join group request without member.name.
> > So
> > > > > this
> > > > > > >     > >>>> coordination
> > > > > > >     > >>>>>> change will be differentiated from the
> > > > > `dynamic-membership`
> > > > > > >     > >> protocol
> > > > > > >     > >>> we
> > > > > > >     > >>>>>> currently have.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When handling static join group request:
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>  1.   The broker will check the membership to see
> > > > whether
> > > > > > this is
> > > > > > >     > >> a
> > > > > > >     > >>>> new
> > > > > > >     > >>>>>> member. If new, broker allocate a unique member id,
> > > > cache
> > > > > > the
> > > > > > >     > >> mapping
> > > > > > >     > >>>> and
> > > > > > >     > >>>>>> move to rebalance stage.
> > > > > > >     > >>>>>>  2.   Following 1, if this is an existing member,
> > > broker
> > > > > > will not
> > > > > > >     > >>>> change
> > > > > > >     > >>>>>> group state, and return its cached member.id and
> > > > current
> > > > > > >     > >> assignment.
> > > > > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > > > > >     > >>>>>>  3.   Although Guozhang has mentioned we could
> > rejoin
> > > > with
> > > > > > pair
> > > > > > >     > >>> member
> > > > > > >     > >>>>>> name and id, I think for join group request it is ok
> > > to
> > > > > > leave
> > > > > > >     > >> member
> > > > > > >     > >>> id
> > > > > > >     > >>>>>> blank as member name is the unique identifier. In
> > > commit
> > > > > > offset
> > > > > > >     > >>> request
> > > > > > >     > >>>>> we
> > > > > > >     > >>>>>> *must* have both.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When handling commit offset request, if enabled with
> > > > > static
> > > > > > >     > >>> membership,
> > > > > > >     > >>>>>> each time the commit request must have both
> > > member.name
> > > > > and
> > > > > > >     > >>> member.id
> > > > > > >     > >>>> to
> > > > > > >     > >>>>>> be identified as a `certificated member`. If not,
> > this
> > > > > > means there
> > > > > > >     > >>> are
> > > > > > >     > >>>>>> duplicate consumer members with same member name and
> > > the
> > > > > > request
> > > > > > >     > >> will
> > > > > > >     > >>>> be
> > > > > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> > > > client
> > > > > > will
> > > > > > >     > >> send a
> > > > > > >     > >>>>>> leave group request (static membership mode). In
> > > static
> > > > > > membership,
> > > > > > >     > >>> we
> > > > > > >     > >>>>> will
> > > > > > >     > >>>>>> also define `change-group-timeout` to hold on
> > > rebalance
> > > > > > provided by
> > > > > > >     > >>>>> leader.
> > > > > > >     > >>>>>> So we will wait for all the members to rejoin the
> > > group
> > > > > and
> > > > > > do
> > > > > > >     > >>> exactly
> > > > > > >     > >>>>> one
> > > > > > >     > >>>>>> rebalance since all members are expected to rejoin
> > > > within
> > > > > > timeout.
> > > > > > >     > >> If
> > > > > > >     > >>>>>> consumer crashes, the join group request from the
> > > > > restarted
> > > > > > >     > >> consumer
> > > > > > >     > >>>> will
> > > > > > >     > >>>>>> be recognized as an existing member and be handled
> > as
> > > > > above
> > > > > > >     > >> condition
> > > > > > >     > >>>> 1;
> > > > > > >     > >>>>>> However, if the consumer takes longer than session
> > > > timeout
> > > > > > to
> > > > > > >     > >> return,
> > > > > > >     > >>>> we
> > > > > > >     > >>>>>> shall still trigger rebalance but it could still try
> > > to
> > > > > > catch
> > > > > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > > > > timeout, its
> > > > > > >     > >>>> cached
> > > > > > >     > >>>>>> state on broker will be garbage collected and
> > trigger
> > > a
> > > > > new
> > > > > > >     > >> rebalance
> > > > > > >     > >>>>> when
> > > > > > >     > >>>>>> it finally joins.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> And consider the switch between dynamic to static
> > > > > > membership.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall
> > revise
> > > > the
> > > > > > >     > >> membership
> > > > > > >     > >>>> to
> > > > > > >     > >>>>>> static and wait for all the current members to
> > > restart,
> > > > > > since their
> > > > > > >     > >>>>>> membership is still dynamic. Here our assumption is
> > > that
> > > > > the
> > > > > > >     > >> restart
> > > > > > >     > >>>>>> process shouldn't take a long time, as long restart
> > is
> > > > > > breaking the
> > > > > > >     > >>>>>> `rebalance timeout` in whatever membership protocol
> > we
> > > > are
> > > > > > using.
> > > > > > >     > >>>> Before
> > > > > > >     > >>>>>> restart, all dynamic member join requests will be
> > > > > rejected.
> > > > > > >     > >>>>>>  2.  Static to dynamic: this is more like a
> > downgrade
> > > > > which
> > > > > > should
> > > > > > >     > >>> be
> > > > > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > > > > session
> > > > > > timeout
> > > > > > >     > >>> to
> > > > > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > > > > current
> > > > > > >     > >>> behavior)
> > > > > > >     > >>>>>>  3.  Halfway switch: a corner case is like some
> > > clients
> > > > > keep
> > > > > > >     > >> dynamic
> > > > > > >     > >>>>>> membership while some keep static membership. This
> > > will
> > > > > > cause the
> > > > > > >     > >>> group
> > > > > > >     > >>>>>> rebalance forever without progress because
> > > > dynamic/static
> > > > > > states
> > > > > > >     > >> are
> > > > > > >     > >>>>>> bouncing each other. This could guarantee that we
> > will
> > > > not
> > > > > > make the
> > > > > > >     > >>>>>> consumer group work in a wrong state by having half
> > > > static
> > > > > > and half
> > > > > > >     > >>>>> dynamic.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> To guarantee correctness, we will also push the
> > member
> > > > > > name/id pair
> > > > > > >     > >>> to
> > > > > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out)
> > and
> > > > > > upgrade the
> > > > > > >     > >> API
> > > > > > >     > >>>>>> version, these details will be further discussed
> > back
> > > in
> > > > > > the KIP.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Are there any concern for this high level proposal?
> > > Just
> > > > > > want to
> > > > > > >     > >>>>> reiterate
> > > > > > >     > >>>>>> on the core idea of the KIP: "If the broker
> > recognize
> > > > this
> > > > > > consumer
> > > > > > >     > >>> as
> > > > > > >     > >>>> an
> > > > > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Thanks a lot for everyone's input! I feel this
> > > proposal
> > > > is
> > > > > > much
> > > > > > >     > >> more
> > > > > > >     > >>>>>> robust than previous one!
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Best,
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Boyang
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> ________________________________
> > > > > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > > > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > > > > >     > >>>>>> To: dev@kafka.apache.org
> > > > > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple
> > > consumer
> > > > > > rebalances
> > > > > > >     > >>> by
> > > > > > >     > >>>>>> specifying member id
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Hi,
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> > > > about
> > > > > > internals
> > > > > > >     > >>>> again
> > > > > > >     > >>>>>> :)
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> I like the idea or a user config `member.name` and
> > to
> > > > > keep
> > > > > > `
> > > > > > >     > >>> member.id`
> > > > > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > > > > client.id`
> > > > > > might
> > > > > > >     > >>> not
> > > > > > >     > >>>>>> be a good idea.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> To clarify the algorithm, each time we generate a
> > new
> > > `
> > > > > > member.id`,
> > > > > > >     > >>> we
> > > > > > >     > >>>>>> also need to update the "group membership"
> > information
> > > > > (ie,
> > > > > > mapping
> > > > > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `
> > > member.id
> > > > `
> > > > > > replaces
> > > > > > >     > >>> the
> > > > > > >     > >>>>>> old entry in the cache.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> I also think, we need to preserve the `member.name
> > ->
> > > > > > member.id`
> > > > > > >     > >>>> mapping
> > > > > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should
> > > mention
> > > > > > this IMHO.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> For changing the default value of config
> > > > > > `leave.group.on.close`. I
> > > > > > >     > >>>> agree
> > > > > > >     > >>>>>> with John, that we should not change the default
> > > config,
> > > > > > because it
> > > > > > >     > >>>>>> would impact all consumer groups with dynamic
> > > > assignment.
> > > > > > However,
> > > > > > >     > >> I
> > > > > > >     > >>>>>> think we can document, that if static assignment is
> > > used
> > > > > > (ie,
> > > > > > >     > >>>>>> `member.name` is configured) we never send a
> > > > > > LeaveGroupRequest
> > > > > > >     > >>>>>> regardless of the config. Note, that the config is
> > > > > > internal, so not
> > > > > > >     > >>>> sure
> > > > > > >     > >>>>>> how to document this in detail. We should not expose
> > > the
> > > > > > internal
> > > > > > >     > >>>> config
> > > > > > >     > >>>>>> in the docs.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> About upgrading: why do we need have two rolling
> > > bounces
> > > > > > and encode
> > > > > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> If we upgrade an existing consumer group from
> > dynamic
> > > to
> > > > > > static, I
> > > > > > >     > >>>> don't
> > > > > > >     > >>>>>> see any reason why both should not work together and
> > > > > single
> > > > > > rolling
> > > > > > >     > >>>>>> bounce would not be sufficient? If we bounce the
> > first
> > > > > > consumer and
> > > > > > >     > >>>>>> switch from dynamic to static, it sends a `
> > > member.name`
> > > > > > and the
> > > > > > >     > >>> broker
> > > > > > >     > >>>>>> registers the [member.name, member.id] in the
> > cache.
> > > > Why
> > > > > > would
> > > > > > >     > >> this
> > > > > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > > > > assignment?
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Also, Guozhang mentioned that for all other request,
> > > we
> > > > > > need to
> > > > > > >     > >> check
> > > > > > >     > >>>> if
> > > > > > >     > >>>>>> the mapping [member.name, member.id] contains the
> > > send
> > > > `
> > > > > > member.id`
> > > > > > >     > >>> --
> > > > > > >     > >>>> I
> > > > > > >     > >>>>>> don't think this is necessary -- it seems to be
> > > > sufficient
> > > > > > to check
> > > > > > >     > >>> the
> > > > > > >     > >>>>>> `member.id` from the [member.id, Assignment]
> > mapping
> > > as
> > > > > be
> > > > > > do
> > > > > > >     > >> today
> > > > > > >     > >>> --
> > > > > > >     > >>>>>> thus, checking `member.id` does not require any
> > > change
> > > > > > IMHO.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> -Matthias
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > > > > >     > >>>>>>> @James
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> What you described is true: the transition from
> > > dynamic
> > > > > to
> > > > > > static
> > > > > > >     > >>>>>>> memberships are not thought through yet. But I do
> > not
> > > > > > think it is
> > > > > > >     > >>> an
> > > > > > >     > >>>>>>> impossible problem: note that we indeed moved the
> > > > offset
> > > > > > commit
> > > > > > >     > >>> from
> > > > > > >     > >>>> ZK
> > > > > > >     > >>>>>> to
> > > > > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is
> > > to
> > > > > > first to
> > > > > > >     > >>>>>>> double-commits on both zk and coordinator, and then
> > > do
> > > > a
> > > > > > second
> > > > > > >     > >>> round
> > > > > > >     > >>>>> to
> > > > > > >     > >>>>>>> turn the zk off.
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > > > > >     > >>>> two-rolling-bounce
> > > > > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag
> > > to
> > > > > > "static"
> > > > > > >     > >>> while
> > > > > > >     > >>>>>> keep
> > > > > > >     > >>>>>>> the registry-id field empty still, in this case,
> > the
> > > > > > coordinator
> > > > > > >     > >>>> still
> > > > > > >     > >>>>>>> follows the logic of "dynamic", accepting the
> > request
> > > > > while
> > > > > > >     > >>> allowing
> > > > > > >     > >>>>> the
> > > > > > >     > >>>>>>> protocol to be set to "static"; after the first
> > > rolling
> > > > > > bounce,
> > > > > > >     > >> the
> > > > > > >     > >>>>> group
> > > > > > >     > >>>>>>> protocol is already "static", then a second rolling
> > > > > bounce
> > > > > > is
> > > > > > >     > >>>> triggered
> > > > > > >     > >>>>>> and
> > > > > > >     > >>>>>>> this time we set the registry-id.
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> Guozhang
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > > > > >     > >> wushujames@gmail.com>
> > > > > > >     > >>>>>> wrote:
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> > > > this:
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > >     > >>>
> > > > > > >     > >>>>> wrote:
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with
> > additional
> > > > > > fields:
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> > > > membership
> > > > > > >     > >>> protocols.
> > > > > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > > > > pre-defined
> > > > > > >     > >>> member
> > > > > > >     > >>>>> id.
> > > > > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> > > > optional
> > > > > > >     > >>>>>>>>> "group-change-timeout" value.
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the
> > > two
> > > > > > protocols
> > > > > > >     > >>> for
> > > > > > >     > >>>>> all
> > > > > > >     > >>>>>>>>> group members: we accept the protocol on the
> > first
> > > > > joined
> > > > > > >     > >> member
> > > > > > >     > >>> of
> > > > > > >     > >>>>> the
> > > > > > >     > >>>>>>>>> group, and if later joining members indicate a
> > > > > different
> > > > > > >     > >>> membership
> > > > > > >     > >>>>>>>>> protocol, we reject it. If the
> > group-change-timeout
> > > > > > value was
> > > > > > >     > >>>>> different
> > > > > > >     > >>>>>>>> to
> > > > > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > > > > application that
> > > > > > >     > >>>> wants
> > > > > > >     > >>>>>> to
> > > > > > >     > >>>>>>>> switch to using static membership? Let's say there
> > > are
> > > > > 10
> > > > > > >     > >>> instances
> > > > > > >     > >>>> of
> > > > > > >     > >>>>>> it.
> > > > > > >     > >>>>>>>> As the instances go through a rolling restart,
> > they
> > > > will
> > > > > > switch
> > > > > > >     > >>> from
> > > > > > >     > >>>>>>>> dynamic membership (the default?) to static
> > > > membership.
> > > > > > As each
> > > > > > >     > >>> one
> > > > > > >     > >>>>>> leaves
> > > > > > >     > >>>>>>>> the group and restarts, they will be rejected from
> > > the
> > > > > > group
> > > > > > >     > >>>> (because
> > > > > > >     > >>>>>> the
> > > > > > >     > >>>>>>>> group is currently using dynamic membership). The
> > > > group
> > > > > > will
> > > > > > >     > >>> shrink
> > > > > > >     > >>>>> down
> > > > > > >     > >>>>>>>> until there is 1 node handling all the traffic.
> > > After
> > > > > > that one
> > > > > > >     > >>>>> restarts,
> > > > > > >     > >>>>>>>> the group will switch over to static membership.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> Is that right? That means that the transition plan
> > > > from
> > > > > > dynamic
> > > > > > >     > >> to
> > > > > > >     > >>>>>> static
> > > > > > >     > >>>>>>>> membership isn't very smooth.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> > > > This
> > > > > > reminds
> > > > > > >     > >> me
> > > > > > >     > >>>> of
> > > > > > >     > >>>>>> the
> > > > > > >     > >>>>>>>> transition plans that were discussed for moving
> > from
> > > > > > >     > >>> zookeeper-based
> > > > > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers.
> > That
> > > > was
> > > > > > also
> > > > > > >     > >>> hard,
> > > > > > >     > >>>>> and
> > > > > > >     > >>>>>>>> ultimately we decided not to build that.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> -James
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> --
> > > > > > >     > >>>> -- Guozhang
> > > > > > >     > >>>>
> > > > > > >     > >>>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> --
> > > > > > >     > >> -- Guozhang
> > > > > > >     > >>
> > > > > > >     >
> > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> >

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Colin McCabe <cm...@apache.org>.
As Jason said, there are definitely scenarios where we know how many group members we expect ahead of time.  It would be nice if we could distinguish between the error case of "we expected 5 clients in the group, but one failed" and a case like "4 clients started up quickly but the 5th took an extra 2 seconds."  We can sandbag the group rebalance delay, but that's a hack which has clear disadvantages.

It would also be nice to be able to detect when a group member left the group briefly but then came back.

I think both of these issues could be solved by having some broker-side metadata about groups which is configured through the admin client.  If there was an "expected group size," stored on the broker-side, then we could rebalance immediately whenever the group size reached that size.  Otherwise, we could apply the rebalance delay, like now.  This would give lower latency when setting things up.

Expected group size is just an expectation, so the group would be allowed to get bigger than that.  We could also have another number which was the maximum group size.  This really would be a hard upper limit on the size of the group, which admins could optionally configure.

When a new client joined a group, the server could send back a unique random 64-bit member ID.  The client could hold on to this ID and use it whenever it rejoined the group after a failure.  Since the ID is random and provided by the server, it can't be spoofed or accidentally reused by a misconfigured client.

best,
Colin

On Fri, Nov 16, 2018, at 00:04, Jason Gustafson wrote:
> >
> > If we initialize a set of member names (I assume ids = names here) on
> > broker through Admin API, the client needs to pick up this information
> > simultaneously which I doubt if there is a generic way to achieve that? It
> > would also make the scaling operations difficult if we need to define the
> > member names every time we change the member set which is an extra
> > operation burden. From my daily ops experience, dynamically generate member
> > names on client side would be easier. Is there a good approach to address
> > this issue?
> 
> 
> Yeah, that's a good question. I'm hoping someone with more kubernetes
> experience will jump in here. Basically my goal is to have an approach
> which maps nicely to StatefulSets (
> https://kubernetes.io/docs/tutorials/stateful-application/basic-stateful-set/).
> The pods in a stateful set have an ordinal index, which sounds similar to
> the static ids that I was describing. You can scale up and down a stateful
> set, but you would need a plugin to grow and shrink the consumer group.
> Sounds like it could work, but I'm not sure if it's the best way.
> 
> At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> > critical streaming services abnormality. One of the burden was the night
> > shift which requires the oncaller to quickly resolve the issue and get the
> > streaming application back on track, however there is a chance of miss. My
> > concern was that if we forfeit the timeout on static membership to trigger
> > rebalance, missing some pages during midnight could be negatively
> > impacting the system performance since we may realize that some partitions
> > stop working for a couple of hours already until next morning. So
> > registration timeout serves as the "last line of defense" to guarantee
> > liveness if no human intervention jumps in.
> 
> 
> Thanks, this is helpful background. I agree this is a risk in the approach
> I've suggested. If we take a step back, I think there are two gaps in the
> protocol for stateful applications:
> 
> 1. We don't have a way to detect the same member across failures or
> restarts. I think streams has some heuristic to try and handle the common
> cases (such as rolling restarts), but the proposal here solves the problem
> in a more robust way.
> 
> 2. We don't have a way to know what the expected membership of the group
> is. This leads us to try tricks like inserting delays into the rebalance
> logic so that the group membership has time to stabilize before we make any
> decisions. In your proposal, we have an expansion timeout, which is
> basically the same thing as far as I can tell.
> 
> I think the first problem is the most important, but it would be nice if we
> can solve the second problem as well. If we have a way to indicate the
> expected group members, then the group can respond to a change much more
> quickly. There would be no need to wait 5 minutes for all members to join
> and it would be robust in the presence of failures. Ironically, static
> membership in this case makes the group more dynamic ;).
> 
> That said, I can see how the registration timeout would be an attractive
> safety net in some cases. Perhaps it would be good enough if we have a way
> to pre-register group members administratively? Members can still be
> expired due to inactivity and we would have a way to get around the
> rebalance delays. Would that work?
> 
> Thanks,
> Jason
> 
> 
> On Wed, Nov 14, 2018 at 10:24 PM, Boyang Chen <bc...@outlook.com> wrote:
> 
> > Thank you for the clarification Jason! The proposals make sense here and
> > let me continue the discussion.
> >
> > > Then the ids would be determined using some convention. Most likely, we
> > would just use sequential numbers 0, 1, 2,
> > > etc. We do the same thing for partition ids.
> >
> >
> > If we initialize a set of member names (I assume ids = names here) on
> > broker through Admin API, the client needs to pick up this information
> > simultaneously which I doubt if there is a generic way to achieve that? It
> > would also make the scaling operations difficult if we need to define the
> > member names every time we change the member set which is an extra
> > operation burden. From my daily ops experience, dynamically generate member
> > names on client side would be easier. Is there a good approach to address
> > this issue?
> >
> > > I was thinking that the registration is specified ahead of time and
> > remains valid until changed. It would be more like a
> > > replica assignment. We don't move partitions just because a broker is
> > down.
> > > The expectation is that it the broker will eventually return.
> >
> >
> > At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> > critical streaming services abnormality. One of the burden was the night
> > shift which requires the oncaller to quickly resolve the issue and get the
> > streaming application back on track, however there is a chance of miss. My
> > concern was that if we forfeit the timeout on static membership to trigger
> > rebalance, missing some pages during midnight could be negatively impacting
> > the system performance since we may realize that some partitions stop
> > working for a couple of hours already until next morning. So registration
> > timeout serves as the "last line of defense" to guarantee liveness if no
> > human intervention jumps in.
> >
> >
> > I'm very interested in the replication protocol currently implemented on
> > Kafka, but I'm not familiar with it. If we do have mechanism to handle
> > issues like I mentioned above for replication (auto healing during
> > mid-night if one broker is never back), we could continue discussing the
> > new approaches to have basic guarantee of consumer group liveness.
> >
> >
> > The discussion so far is to make sure that all the design approaches we
> > have taken are pointing to real scenarios. Once we clarify the scenarios,
> > we would definitely propose better solution on top of it. I hope these
> > discussions make sense. Thanks again for helping make the design solid!
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Jason Gustafson <ja...@confluent.io>
> > Sent: Thursday, November 15, 2018 9:54 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > >
> > > I feel this would make the current protocol harder to use. For example,
> > on
> > > KStream we typically would expect (number of instances * number of
> > threads
> > > per instance) consumers. Giving out all this many member names in a list
> > > may not be easy, compared with dynamic generation of member names, at
> > least
> > > for KStream use case.
> >
> >
> > That's a fair point. What I had in mind is for the API to specify the
> > number of consumers in the group. Then the ids would be determined using
> > some convention. Most likely, we would just use sequential numbers 0, 1, 2,
> > etc. We do the same thing for partition ids. Streams folks can chime in and
> > say whether that would work or not.
> >
> > So this suggests we will not rely on heartbeat and commit interval to
> > > trigger rebalance? Even in static membership, I feel tracking active
> > > members is still required to be handled by broker since not all users are
> > > fully equipped with monitoring tools, otherwise users will feel
> > > uncomfortable using static membership solely for reducing rebalance
> > purpose.
> >
> >
> > Let me clarify. I think it is still useful to distinguish the liveness of
> > the consumer using the heartbeat mechanism and to propagate that
> > information to the leader during rebalances. This gives the group leader
> > the option (but not the requirement) to change the partition assignment for
> > inactive members. My suggestion was more about how long the static
> > registration would remain valid. In the current proposal we have a
> > registration timeout, and I think Mayuresh is fair to point out the
> > potential confusion with the session timeout. It's kind of the same, but
> > not exactly. So instead, I was thinking that the registration is specified
> > ahead of time and remains valid until changed. It would be more like a
> > replica assignment. We don't move partitions just because a broker is down.
> > The expectation is that it the broker will eventually return. Similarly for
> > stateful applications, we would have the option to do the same thing. The
> > registration is fixed until someone changes it.
> >
> > Does that make sense?
> >
> > Thanks,
> > Jason
> >
> >
> > On Wed, Nov 14, 2018 at 2:46 PM, Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Thanks Jason for the suggestions! I update the KIP with full schema
> > > changes.
> > >
> > > > we offer an admin API that lets a user define the expected members of
> > > the group.
> > >
> > > I feel this would make the current protocol harder to use. For example,
> > on
> > > KStream we typically would expect (number of instances * number of
> > threads
> > > per instance) consumers. Giving out all this many member names in a list
> > > may not be easy, compared with dynamic generation of member names, at
> > least
> > > for KStream use case.
> > >
> > > > For the sake of discussion, I was wondering if we could just say that
> > > static members do not expire.
> > >
> > >
> > > So this suggests we will not rely on heartbeat and commit interval to
> > > trigger rebalance? Even in static membership, I feel tracking active
> > > members is still required to be handled by broker since not all users are
> > > fully equipped with monitoring tools, otherwise users will feel
> > > uncomfortable using static membership solely for reducing rebalance
> > purpose.
> > >
> > >
> > > For the admin API design, I'm simplifying the join group request
> > handling,
> > > while using admin tool to switch between static and dynamic membership
> > and
> > > set the two corresponding timeouts. Do you think this approach makes
> > sense?
> > > The version one implementation will be much more clean if we handle
> > > membership change through user intervention.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Jason Gustafson <ja...@confluent.io>
> > > Sent: Wednesday, November 14, 2018 9:31 AM
> > > To: dev
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hey Boyang,
> > >
> > > Thanks for the updates. From a high level, I think this actually
> > > complements Konstantine's writeup on incremental rebalancing. The gap
> > we're
> > > addressing is providing a way to bind the the partition assignment of a
> > > group to a set of user-provided ids so that we are not so reliant on the
> > > group's immediate state. For example, these ids might identify the state
> > > store volume for particular streams instances. This is basically what you
> > > need to work well with k8s stateful sets (as far as I understand them).
> > >
> > > One key decision is how we would define and update the expected static
> > > members in a consumer group. The mechanics of the registration and
> > > expansion timeouts feel a little bit clunky. For the sake of discussion,
> > I
> > > was wondering if we could just say that static members do not expire.
> > > Instead, we offer an admin API that lets a user define the expected
> > members
> > > of the group. This API could be used to both grow and shrink a group.
> > This
> > > would solve the rebalancing problems when applications are initially
> > > bootstrapped or when they are restarted because we would always know how
> > > many members should be in a group. What do you think?
> > >
> > > By the way, it would be helpful to include the full schema definition for
> > > any protocol changes in the proposal.
> > >
> > > Thanks,
> > > Jason
> > >
> > >
> > > On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com>
> > wrote:
> > >
> > > > Thanks Mayuresh for the feedback! Do you have a quick example for
> > passing
> > > > in consumer config dynamically? I mainly use Kafka Streams at my daily
> > > work
> > > > so probably missing the idea how to do it in the current consumer
> > > setting.
> > > >
> > > >
> > > > For differentiating session timeout and registration timeout, I would
> > try
> > > > to enhance the documentation in the first stage to see how people react
> > > to
> > > > the confusion (would be great if they feel straightforward!). Since one
> > > > doesn't have to fully understand the difference unless defining the new
> > > > config "member name", for current users we could buy some time to
> > listen
> > > to
> > > > their understandings and improve our documentation correspondingly in
> > the
> > > > follow-up KIPs.
> > > >
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Mayuresh Gharat <gh...@gmail.com>
> > > > Sent: Sunday, November 11, 2018 1:06 PM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > > specifying member id
> > > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the reply.
> > > >
> > > > Please find the replies inline below :
> > > > For having a consumer config at runtime, I think it's not necessary to
> > > > address in this KIP because most companies run sidecar jobs through
> > > daemon
> > > > software like puppet. It should be easy to change the config through
> > > script
> > > > or UI without actual code change. We still want to leave flexibility
> > for
> > > > user to define member name as they like.
> > > > ---- This might be little different for companies that use
> > configuration
> > > > management tools that does not allow the applications to define/change
> > > the
> > > > configs dynamically. For example, if we use something similar to spring
> > > to
> > > > pull in the configs for the KafkaConsumer and pass it to the
> > constructor
> > > to
> > > > create the KafkaConsumer object, it will be hard to specify a unique
> > > value
> > > > to the "MEMBER_NAME" config unless someone deploying the app generates
> > a
> > > > unique string for this config outside the deployment workflow and
> > copies
> > > it
> > > > statically before starting up each consumer instance. Unless we can
> > > loosen
> > > > the criteria for uniqueness of this config value, for each consumer
> > > > instance in the consumer group, I am not sure of a better way of
> > > > addressing this. If we don't want to loosen the criteria, then
> > providing
> > > a
> > > > dynamic way to pass this in at runtime, would put the onus of having
> > the
> > > > same unique value each time a consumer is restarted, on to the
> > > application
> > > > that is running the consumer.
> > > >
> > > > I just updated the kip about having both "registration timeout" and
> > > > "session timeout". The benefit of having two configs instead of one is
> > to
> > > > reduce the mental burden for operation, for example user just needs to
> > > > unset "member name" to cast back to dynamic membership without worrying
> > > > about tuning the "session timeout" back to a smaller value.
> > > > --- That is a good point. I was thinking, if both the configs are
> > > > specified, it would be confusing for the end user without understanding
> > > the
> > > > internals of the consumer and its interaction with group coordinator,
> > as
> > > > which takes precedence when and how it affects the consumer behavior.
> > > Just
> > > > my 2 cents.
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > > On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > > > Hey Mayuresh,
> > > > >
> > > > >
> > > > > thanks for the thoughtful questions! Let me try to answer your
> > > questions
> > > > > one by one.
> > > > >
> > > > >
> > > > > For having a consumer config at runtime, I think it's not necessary
> > to
> > > > > address in this KIP because most companies run sidecar jobs through
> > > > daemon
> > > > > software like puppet. It should be easy to change the config through
> > > > script
> > > > > or UI without actual code change. We still want to leave flexibility
> > > for
> > > > > user to define member name as they like.
> > > > >
> > > > >
> > > > > I just updated the kip about having both "registration timeout" and
> > > > > "session timeout". The benefit of having two configs instead of one
> > is
> > > to
> > > > > reduce the mental burden for operation, for example user just needs
> > to
> > > > > unset "member name" to cast back to dynamic membership without
> > worrying
> > > > > about tuning the "session timeout" back to a smaller value.
> > > > >
> > > > >
> > > > > For backup topic, I think it's a low-level detail which could be
> > > > addressed
> > > > > in the implementation. I feel no preference of adding a new topic vs
> > > > reuse
> > > > > consumer offsets topic. I will do more analysis and make a trade-off
> > > > > comparison. Nice catch!
> > > > >
> > > > >
> > > > > I hope the explanations make sense to you. I will keep polishing on
> > the
> > > > > edge cases and details.
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Mayuresh Gharat <gh...@gmail.com>
> > > > > Sent: Saturday, November 10, 2018 10:25 AM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Hi Boyang,
> > > > >
> > > > > Thanks for the KIP and sorry for being late to the party. This KIP is
> > > > > really useful for us at Linkedin.
> > > > >
> > > > > I had a few questions :
> > > > >
> > > > > The idea of having static member name seems nice, but instead of a
> > > > config,
> > > > > would it be possible for it to be passed in to the consumer at
> > runtime?
> > > > > This is because an app might want to decide the config value at
> > runtime
> > > > > using its host information for example, to generate the unique member
> > > > name.
> > > > >
> > > > > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > > > > wondering if we can reuse the session timeout here. This might help
> > us
> > > to
> > > > > have one less config on the consumer.
> > > > >
> > > > > The KIP also talks about adding another internal topic
> > > > "static_member_map".
> > > > > Would the semantics (GroupCoordinator broker, topic configs) be the
> > > same
> > > > as
> > > > > __consumer_offsets topic?
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Mayuresh
> > > > >
> > > > >
> > > > > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com>
> > > wrote:
> > > > >
> > > > > > I took a quick pass of the proposal. First I would say it's a very
> > > > > > brilliant initiative from Konstantine and Confluent folks. To draft
> > > up
> > > > a
> > > > > > proposal like this needs deep understanding of the rebalance
> > > protocol!
> > > > I
> > > > > > summarized some thoughts here.
> > > > > >
> > > > > >
> > > > > > Overall the motivations of the two proposals align on that:
> > > > > >
> > > > > >   1.  Both believe the invariant resource (belonging to the same
> > > > process)
> > > > > > should be preserved across rebalance.
> > > > > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > > > > redistribution. I don't use rebalance here since part one of the
> > > > > > cooperative proposal could potentially introduce more rebalances
> > but
> > > > only
> > > > > > on must-move resources.
> > > > > >   3.  Scale up/down and rolling bounce are causing unnecessary
> > > resource
> > > > > > shuffling that need to be mitigated.
> > > > > >
> > > > > >
> > > > > > On motivation level, I think both approach could solve/mitigate the
> > > > above
> > > > > > issues. They are just different in design philosophy, or I would
> > say
> > > > the
> > > > > > perspective difference between framework user and algorithm
> > designer.
> > > > > >
> > > > > >
> > > > > > Two proposals have different focuses. KIP-345 is trying to place
> > more
> > > > > > fine-grained control on the broker side to reduce the unnecessary
> > > > > > rebalances, while keeping the client logic intact. This is pretty
> > > > > intuitive
> > > > > > cause-effect for normal developers who are not very familiar with
> > > > > rebalance
> > > > > > protocol. As a developer working with Kafka Streams daily, I'd be
> > > happy
> > > > > to
> > > > > > see a simplified rebalance protocol and just focus on maintaining
> > the
> > > > > > stream/consumer jobs. Too many rebalances raised my concern on the
> > > job
> > > > > > health. To be concise, static membership has the advantage of
> > > reducing
> > > > > > mental burden.
> > > > > >
> > > > > >
> > > > > > Cooperative proposal takes thoughtful approach on client side. We
> > > want
> > > > to
> > > > > > have fine-grained control on the join/exit group behaviors and make
> > > the
> > > > > > current dynamic membership better to address above issues. I do
> > feel
> > > > our
> > > > > > idea crossed on the delayed rebalance when we scale up/down, which
> > > > could
> > > > > > potentially reduce the state shuffling and decouple the behavior
> > from
> > > > > > session timeout which is already overloaded.  In this sense, I
> > > believe
> > > > > both
> > > > > > approaches would serve well in making "reasonable rebalance" happen
> > > at
> > > > > the
> > > > > > "right timing".
> > > > > >
> > > > > >
> > > > > > However, based on my understanding, either 345 or cooperative
> > > > rebalancing
> > > > > > is not solving the problem Mike has proposed: could we do a better
> > > job
> > > > at
> > > > > > scaling up/down in ideal timing? My initial response was to
> > introduce
> > > > an
> > > > > > admin API which now I feel is sub-optimal, in that the goal of
> > smooth
> > > > > > transition is to make sure the newly up hosts are actually "ready".
> > > For
> > > > > > example:
> > > > > >
> > > > > >
> > > > > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At
> > > some
> > > > > > time we would like to scale up to 8 hosts, with the current
> > > > improvements
> > > > > we
> > > > > > could reduce 4 potential rebalances to a single one. But the new
> > > hosts
> > > > > are
> > > > > > yet unknown to be "ready" if they need to reconstruct the local
> > > state.
> > > > To
> > > > > > be actually ready, we need 4 standby tasks running on those empty
> > > hosts
> > > > > and
> > > > > > leader needs to wait for the signal of "replay/reconstruct
> > complete"
> > > to
> > > > > > actually involve them into the main consumer group. Otherwise,
> > > > rebalance
> > > > > > just kills our performance since we need to wait indefinite long
> > for
> > > > task
> > > > > > migration.
> > > > > >
> > > > > >
> > > > > > The scale down is also tricky such that we are not able to define a
> > > > > "true"
> > > > > > leave of a member. Rebalance immediately after "true" leaves are
> > most
> > > > > > optimal comparing with human intervention. Does this make sense?
> > > > > >
> > > > > >
> > > > > > My intuition is that cooperative approach which was implemented on
> > > the
> > > > > > client side could better handle scaling cases than KIP 345, since
> > it
> > > > > > involves a lot of algorithmic changes to define "replaying" stage,
> > > > which
> > > > > I
> > > > > > feel would over-complicate broker logic if implemented on
> > > coordinator.
> > > > If
> > > > > > we let 345 focus on reducing unnecessary rebalance, and let
> > > cooperative
> > > > > > approach focus on judging best timing of scale up/down, the two
> > > efforts
> > > > > > could be aligned. In long term, I feel the more complex improvement
> > > of
> > > > > > consumer protocol should happen on client side instead of server
> > side
> > > > > which
> > > > > > is easier to test and has less global impact for the entire Kafka
> > > > > > production cluster.
> > > > > >
> > > > > >
> > > > > > Thanks again to Konstantine, Matthias and other folks in coming up
> > > with
> > > > > > this great client proposal. This is great complementation to KIP
> > 345.
> > > > In
> > > > > a
> > > > > > high level, we are not having any collision on the path and both
> > > > > proposals
> > > > > > are making sense here. Just need better sync to avoid duplicate
> > > effort
> > > > :)
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > >
> > > > > > ________________________________
> > > > > > From: Boyang Chen <bc...@outlook.com>
> > > > > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > > > > To: dev@kafka.apache.org
> > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > > by
> > > > > > specifying member id
> > > > > >
> > > > > > Thanks Matthias for bringing this awesome proposal up! I shall
> > take a
> > > > > > deeper look and make a comparison between the two proposals.
> > > > > >
> > > > > >
> > > > > > Meanwhile for the scale down specifically for stateful streaming,
> > we
> > > > > could
> > > > > > actually introduce a new status called "learner" where the newly up
> > > > hosts
> > > > > > could try to catch up with the assigned task progress first before
> > > > > > triggering the rebalance, from which we don't see a sudden dip on
> > the
> > > > > > progress. However, it is built on top of the success of KIP-345.
> > > > > >
> > > > > >
> > > > > > ________________________________
> > > > > > From: Matthias J. Sax <ma...@confluent.io>
> > > > > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > > > > To: dev@kafka.apache.org
> > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > > by
> > > > > > specifying member id
> > > > > >
> > > > > > Hey,
> > > > > >
> > > > > > there was quite a pause on this KIP discussion and in the mean
> > time,
> > > a
> > > > > > new design for incremental cooporative rebalance was suggested:
> > > > > >
> > > > > >
> > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Increm
> > > > ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> > > > cwiki.apache.org
> > > > Rebalancing between distributed application processes in Apache Kafka
> > was
> > > > enhanced considerably when it was decoupled as logic from Kafka brokers
> > > and
> > > > was moved as responsibility to the clients and specifically to Kafka
> > > > Consumer. This pattern has been working robustly for quite a while now
> > > and
> > > > has ...
> > > >
> > > >
> > > >
> > > > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > > > >
> > > > > cwiki.apache.org
> > > > > Rebalancing between distributed application processes in Apache Kafka
> > > was
> > > > > enhanced considerably when it was decoupled as logic from Kafka
> > brokers
> > > > and
> > > > > was moved as responsibility to the clients and specifically to Kafka
> > > > > Consumer. This pattern has been working robustly for quite a while
> > now
> > > > and
> > > > > has ...
> > > > >
> > > > >
> > > > >
> > > > > >
> > > > > >
> > > > > > We should make sure that the proposal and this KIP align to each
> > > other.
> > > > > > Thoughts?
> > > > > >
> > > > > >
> > > > > > -Matthias
> > > > > >
> > > > > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > > > > Hey Mike,
> > > > > > >
> > > > > > >
> > > > > > > thanks for the feedback, the two question are very thoughtful!
> > > > > > >
> > > > > > >
> > > > > > >> 1) I am a little confused about the distinction for the leader.
> > If
> > > > the
> > > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > > quickly
> > > > > > comes up) to update application code, will a rebalance be
> > triggered?
> > > I
> > > > >
> > > > > do
> > > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > > >
> > > > > > > For Q1 my intention was to minimize the change within one KIP,
> > > since
> > > > > the
> > > > > > leader rejoining case could be addressed separately.
> > > > > > >
> > > > > > >
> > > > > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > > gracefully increase the number of nodes in the cluster. I think we
> > > need
> > > > > to
> > > > > > support graceful shrink down as well. If I set the registration
> > > timeout
> > > > > to
> > > > > > 5 minutes > to handle rolling restarts or intermittent failures
> > > without
> > > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > > group
> > > > to
> > > > > > rebalance if I am intentionally removing a node from the cluster. I
> > > am
> > > > > not
> > > > > > sure the best way to > do this. One idea I had was adding the
> > ability
> > > > > for a
> > > > > > CLI or Admin API to force a rebalance of the group. This would
> > allow
> > > > for
> > > > > an
> > > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > > registration timeout on > shrink down. What do you think?
> > > > > > >
> > > > > > > For 2) my understanding is that for scaling down case it is
> > better
> > > to
> > > > > be
> > > > > > addressed by CLI tool than code logic, since only by human
> > evaluation
> > > > we
> > > > > > could decide whether it is a "right timing" -- the time when all
> > the
> > > > > > scaling down consumers are offline -- to kick in rebalance. Unless
> > we
> > > > > > introduce another term on coordinator which indicates the target
> > > > consumer
> > > > > > group size, broker will find it hard to decide when to start
> > > rebalance.
> > > > > So
> > > > > > far I prefer to hold the implementation for that, but agree we
> > could
> > > > > > discuss whether we want to introduce admin API in this KIP or a
> > > > separate
> > > > > > one.
> > > > > > >
> > > > > > >
> > > > > > > Thanks again for the proposed ideas!
> > > > > > >
> > > > > > >
> > > > > > > Boyang
> > > > > > >
> > > > > > > ________________________________
> > > > > > > From: Mike Freyberger <mi...@xandr.com>
> > > > > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > > > > To: dev@kafka.apache.org
> > > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by
> > > > > > specifying member id
> > > > > > >
> > > > > > > Boyang,
> > > > > > >
> > > > > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > > > > >
> > > > > > > 1) I am a little confused about the distinction for the leader.
> > If
> > > > the
> > > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > > quickly
> > > > > > comes up) to update application code, will a rebalance be
> > triggered?
> > > I
> > > > do
> > > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > > >
> > > > > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > > gracefully increase the number of nodes in the cluster. I think we
> > > need
> > > > > to
> > > > > > support graceful shrink down as well. If I set the registration
> > > timeout
> > > > > to
> > > > > > 5 minutes to handle rolling restarts or intermittent failures
> > without
> > > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > > group
> > > > to
> > > > > > rebalance if I am intentionally removing a node from the cluster. I
> > > am
> > > > > not
> > > > > > sure the best way to do this. One idea I had was adding the ability
> > > > for a
> > > > > > CLI or Admin API to force a rebalance of the group. This would
> > allow
> > > > for
> > > > > an
> > > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > > registration timeout on shrink down. What do you think?
> > > > > > >
> > > > > > > Mike
> > > > > > >
> > > > > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com>
> > wrote:
> > > > > > >
> > > > > > >     Btw, I updated KIP 345 based on my understanding. Feel free
> > to
> > > > take
> > > > > > another round of look:
> > > > > > >
> > > > > > >
> > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > > KIP-345: Introduce static membership protocol to reduce ...<
> > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > >
> > > > > > > cwiki.apache.org
> > > > > > > For stateful applications, one of the biggest performance
> > > bottleneck
> > > > is
> > > > > > the state shuffling. In Kafka consumer, there is a concept called
> > > > > > "rebalance" which means that for given M partitions and N consumers
> > > in
> > > > > one
> > > > > > consumer group, Kafka will try to balance the load between
> > consumers
> > > > and
> > > > > > ideally have ...
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > > > > >
> > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > >
> > > > > > >     cwiki.apache.org
> > > > > > >     For stateful applications, one of the biggest performance
> > > > > bottleneck
> > > > > > is the state shuffling. In Kafka consumer, there is a concept
> > called
> > > > > > "rebalance" which means that for given M partitions and N consumers
> > > in
> > > > > one
> > > > > > consumer group, Kafka will try to balance the load between
> > consumers
> > > > and
> > > > > > ideally have ...
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     ________________________________
> > > > > > >     From: Boyang Chen <bc...@outlook.com>
> > > > > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > > > > >     To: dev@kafka.apache.org
> > > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > > by specifying member id
> > > > > > >
> > > > > > >     Thanks everyone for the input on this thread! (Sorry it's
> > been
> > > a
> > > > > > while) I feel that we are very close to the final solution.
> > > > > > >
> > > > > > >
> > > > > > >     Hey Jason and Mike, I have two quick questions on the new
> > > > features
> > > > > > here:
> > > > > > >
> > > > > > >       1.  so our proposal is that until we add a new static
> > member
> > > > into
> > > > > > the group (scale up), we will not trigger rebalance until the
> > > > > "registration
> > > > > > timeout"( the member has been offline for too long)? How about
> > > leader's
> > > > > > rejoin request, I think we should still trigger rebalance when that
> > > > > > happens, since the consumer group may have new topics to consume?
> > > > > > >       2.  I'm not very clear on the scale up scenario in static
> > > > > > membership here. Should we fallback to dynamic membership while
> > > > > > adding/removing hosts (by setting member.name = null), or we still
> > > > want
> > > > > > to add instances with `member.name` so that we eventually
> > > > expand/shrink
> > > > > > the static membership? I personally feel the easier solution is to
> > > spin
> > > > > up
> > > > > > new members and wait until either the same "registration timeout"
> > or
> > > a
> > > > > > "scale up timeout" before starting the rebalance. What do you
> > think?
> > > > > > >
> > > > > > >     Meanwhile I will go ahead to make changes to the KIP with our
> > > > newly
> > > > > > discussed items and details. Really excited to see the design has
> > > > become
> > > > > > more solid.
> > > > > > >
> > > > > > >     Best,
> > > > > > >     Boyang
> > > > > > >
> > > > > > >     ________________________________
> > > > > > >     From: Jason Gustafson <ja...@confluent.io>
> > > > > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > > > > >     To: dev
> > > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > > rebalances
> > > > > > by specifying member id
> > > > > > >
> > > > > > >     Hey Mike,
> > > > > > >
> > > > > > >     Yeah, that's a good point. A long "registration timeout" may
> > > not
> > > > be
> > > > > > a great
> > > > > > >     idea. Perhaps in practice you'd set it long enough to be able
> > > to
> > > > > > detect a
> > > > > > >     failure and provision a new instance. Maybe on the order of
> > 10
> > > > > > minutes is
> > > > > > >     more reasonable.
> > > > > > >
> > > > > > >     In any case, it's probably a good idea to have an
> > > administrative
> > > > > way
> > > > > > to
> > > > > > >     force deregistration. One option is to extend the
> > DeleteGroups
> > > > API
> > > > > > with a
> > > > > > >     list of members names.
> > > > > > >
> > > > > > >     -Jason
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > > > > mfreyberger@appnexus.com>
> > > > > > >     wrote:
> > > > > > >
> > > > > > >     > Jason,
> > > > > > >     >
> > > > > > >     > Regarding step 4 in your proposal which suggests beginning
> > a
> > > > long
> > > > > > timer
> > > > > > >     > (30 minutes) when a static member leaves the group, would
> > > there
> > > > > > also be the
> > > > > > >     > ability for an admin to force a static membership
> > expiration?
> > > > > > >     >
> > > > > > >     > I'm thinking that during particular types of outages or
> > > > upgrades
> > > > > > users
> > > > > > >     > would want forcefully remove a static member from the
> > group.
> > > > > > >     >
> > > > > > >     > So the user would shut the consumer down normally, which
> > > > wouldn't
> > > > > > trigger
> > > > > > >     > a rebalance. Then the user could use an admin CLI tool to
> > > force
> > > > > > remove that
> > > > > > >     > consumer from the group, so the TopicPartitions that were
> > > > > > previously owned
> > > > > > >     > by that consumer can be released.
> > > > > > >     >
> > > > > > >     > At a high level, we need consumer groups to gracefully
> > handle
> > > > > > intermittent
> > > > > > >     > failures and permanent failures. Currently, the consumer
> > > group
> > > > > > protocol
> > > > > > >     > handles permanent failures well, but does not handle
> > > > intermittent
> > > > > > failures
> > > > > > >     > well (it creates unnecessary rebalances). I want to make
> > sure
> > > > the
> > > > > > overall
> > > > > > >     > solution here handles both intermittent failures and
> > > permanent
> > > > > > failures,
> > > > > > >     > rather than sacrificing support for permanent failures in
> > > order
> > > > > to
> > > > > > provide
> > > > > > >     > support for intermittent failures.
> > > > > > >     >
> > > > > > >     > Mike
> > > > > > >     >
> > > > > > >     > Sent from my iPhone
> > > > > > >     >
> > > > > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > > wrote:
> > > > > > >     > >
> > > > > > >     > > Hey Guozhang,
> > > > > > >     > >
> > > > > > >     > > Responses below:
> > > > > > >     > >
> > > > > > >     > > Originally I was trying to kill more birds with one stone
> > > > with
> > > > > > KIP-345,
> > > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > > shutting
> > > > > > down a
> > > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> > early
> > > > > > email), and
> > > > > > >     > >> hence proposing to have a pure static-membership
> > protocol.
> > > > But
> > > > > > thinking
> > > > > > >     > >> twice about it I now feel it may be too ambitious and
> > > worth
> > > > > > fixing in
> > > > > > >     > >> another KIP.
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > I was considering an extension to support
> > > pre-initialization
> > > > of
> > > > > > the
> > > > > > >     > static
> > > > > > >     > > members of the group, but I agree we should probably
> > leave
> > > > this
> > > > > > problem
> > > > > > >     > for
> > > > > > >     > > future work.
> > > > > > >     > >
> > > > > > >     > > 1. How this longish static member expiration timeout
> > > defined?
> > > > > Is
> > > > > > it via a
> > > > > > >     > >> broker, hence global config, or via a client config
> > which
> > > > can
> > > > > be
> > > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > I am not too sure. I tend to lean toward server-side
> > > configs
> > > > > > because they
> > > > > > >     > > are easier to evolve. If we have to add something to the
> > > > > > protocol, then
> > > > > > >     > > we'll be stuck with it forever.
> > > > > > >     > >
> > > > > > >     > > 2. Assuming that for static members, LEAVE_GROUP request
> > > will
> > > > > not
> > > > > > >     > trigger a
> > > > > > >     > >> rebalance immediately either, similar to session
> > timeout,
> > > > but
> > > > > > only the
> > > > > > >     > >> longer member expiration timeout, can we remove the
> > > > internal "
> > > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > > walk-around
> > > > > > >     > then?
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > > Yeah, I hope we can ultimately get rid of it, but we may
> > > need
> > > > > it
> > > > > > for
> > > > > > >     > > compatibility with older brokers. A related question is
> > > what
> > > > > > should be
> > > > > > >     > the
> > > > > > >     > > behavior of the consumer if `member.name` is provided
> > but
> > > > the
> > > > > > broker
> > > > > > >     > does
> > > > > > >     > > not support it? We could either fail or silently
> > downgrade
> > > to
> > > > > > dynamic
> > > > > > >     > > membership.
> > > > > > >     > >
> > > > > > >     > > -Jason
> > > > > > >     > >
> > > > > > >     > >
> > > > > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > >     > wrote:
> > > > > > >     > >>
> > > > > > >     > >> Hey Jason,
> > > > > > >     > >>
> > > > > > >     > >> I like your idea to simplify the upgrade protocol to
> > allow
> > > > > > co-exist of
> > > > > > >     > >> static and dynamic members. Admittedly it may make the
> > > > > > coordinator-side
> > > > > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > > > > >     > >>
> > > > > > >     > >> Originally I was trying to kill more birds with one
> > stone
> > > > with
> > > > > > KIP-345,
> > > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > > shutting
> > > > > > down a
> > > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> > early
> > > > > > email), and
> > > > > > >     > >> hence proposing to have a pure static-membership
> > protocol.
> > > > But
> > > > > > thinking
> > > > > > >     > >> twice about it I now feel it may be too ambitious and
> > > worth
> > > > > > fixing in
> > > > > > >     > >> another KIP. With that, I think what you've proposed
> > here
> > > > is a
> > > > > > good way
> > > > > > >     > to
> > > > > > >     > >> go for KIP-345 itself.
> > > > > > >     > >>
> > > > > > >     > >> Note there are a few details in your proposal we'd still
> > > > need
> > > > > > to figure
> > > > > > >     > >> out:
> > > > > > >     > >>
> > > > > > >     > >> 1. How this longish static member expiration timeout
> > > > defined?
> > > > > > Is it via
> > > > > > >     > a
> > > > > > >     > >> broker, hence global config, or via a client config
> > which
> > > > can
> > > > > be
> > > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > > >     > >>
> > > > > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> > > > will
> > > > > > not
> > > > > > >     > trigger a
> > > > > > >     > >> rebalance immediately either, similar to session
> > timeout,
> > > > but
> > > > > > only the
> > > > > > >     > >> longer member expiration timeout, can we remove the
> > > > internal "
> > > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > > walk-around
> > > > > > >     > then?
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> Guozhang
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > > > > jason@confluent.io>
> > > > > > >     > >> wrote:
> > > > > > >     > >>
> > > > > > >     > >>> Hey All,
> > > > > > >     > >>>
> > > > > > >     > >>> Nice to see some solid progress on this. It sounds like
> > > one
> > > > > of
> > > > > > the
> > > > > > >     > >>> complications is allowing static and dynamic
> > registration
> > > > to
> > > > > > coexist.
> > > > > > >     > I'm
> > > > > > >     > >>> wondering if we can do something like the following:
> > > > > > >     > >>>
> > > > > > >     > >>> 1. Statically registered members (those joining the
> > group
> > > > > with
> > > > > > a
> > > > > > >     > >> non-null `
> > > > > > >     > >>> member.name`) maintain a session with the coordinator
> > > just
> > > > > > like
> > > > > > >     > dynamic
> > > > > > >     > >>> members.
> > > > > > >     > >>> 2. If a session is active for a static member when a
> > > > > rebalance
> > > > > > begins,
> > > > > > >     > >> then
> > > > > > >     > >>> basically we'll keep the current behavior. The
> > rebalance
> > > > will
> > > > > > await the
> > > > > > >     > >>> static member joining the group.
> > > > > > >     > >>> 3. If a static member does not have an active session,
> > > then
> > > > > the
> > > > > > >     > >> coordinator
> > > > > > >     > >>> will not wait for it to join, but will still include it
> > > in
> > > > > the
> > > > > > >     > rebalance.
> > > > > > >     > >>> The coordinator will forward the cached subscription
> > > > > > information to the
> > > > > > >     > >>> leader and will cache the assignment after the
> > rebalance
> > > > > > completes.
> > > > > > >     > (Note
> > > > > > >     > >>> that we still have the generationId to fence offset
> > > commits
> > > > > > from a
> > > > > > >     > static
> > > > > > >     > >>> zombie if the assignment changes.)
> > > > > > >     > >>> 4. When a static member leaves the group or has its
> > > session
> > > > > > expire, no
> > > > > > >     > >>> rebalance is triggered. Instead, we can begin a timer
> > to
> > > > > > expire the
> > > > > > >     > >> static
> > > > > > >     > >>> registration. This would be a longish timeout (like 30
> > > > > minutes
> > > > > > say).
> > > > > > >     > >>>
> > > > > > >     > >>> So basically static members participate in all
> > rebalances
> > > > > > regardless
> > > > > > >     > >>> whether they have an active session. In a given
> > > rebalance,
> > > > > > some of the
> > > > > > >     > >>> members may be static and some dynamic. The group
> > leader
> > > > can
> > > > > > >     > >> differentiate
> > > > > > >     > >>> the two based on the presence of the `member.name` (we
> > > > have
> > > > > > to add
> > > > > > >     > this
> > > > > > >     > >> to
> > > > > > >     > >>> the JoinGroupResponse). Generally speaking, we would
> > > choose
> > > > > > leaders
> > > > > > >     > >>> preferentially from the active members that support the
> > > > > latest
> > > > > > >     > JoinGroup
> > > > > > >     > >>> protocol and are using static membership. If we have to
> > > > > choose
> > > > > > a leader
> > > > > > >     > >>> with an old version, however, it would see all members
> > in
> > > > the
> > > > > > group
> > > > > > >     > >> (static
> > > > > > >     > >>> or dynamic) as dynamic members and perform the
> > assignment
> > > > as
> > > > > > usual.
> > > > > > >     > >>>
> > > > > > >     > >>> Would that work?
> > > > > > >     > >>>
> > > > > > >     > >>> -Jason
> > > > > > >     > >>>
> > > > > > >     > >>>
> > > > > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com>
> > > > > > >     > >> wrote:
> > > > > > >     > >>>
> > > > > > >     > >>>> Hello Boyang,
> > > > > > >     > >>>>
> > > > > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > > > > >     > >>>>
> > > > > > >     > >>>> 1. Where will "change-group-timeout" be communicated
> > to
> > > > the
> > > > > > broker?
> > > > > > >     > >> Will
> > > > > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> > > > going
> > > > > > to
> > > > > > >     > >>> piggy-back
> > > > > > >     > >>>> on the existing session-timeout field (assuming that
> > the
> > > > > > original
> > > > > > >     > value
> > > > > > >     > >>>> will not be used anywhere in the static membership any
> > > > > more)?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > > > > timeout to
> > > > > > >     > >> return,
> > > > > > >     > >>>> we shall still trigger rebalance but it could still
> > try
> > > to
> > > > > > catch
> > > > > > >     > >>>> `change-group-timeout`.": what does this mean? I
> > thought
> > > > > your
> > > > > > proposal
> > > > > > >     > >> is
> > > > > > >     > >>>> that for static memberships, the broker will NOT
> > trigger
> > > > > > rebalance
> > > > > > >     > even
> > > > > > >     > >>>> after session-timeout has been detected, but only that
> > > > after
> > > > > > >     > >>>> change-group-timeout
> > > > > > >     > >>>> which is supposed to be longer than session-timeout to
> > > be
> > > > > > defined?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 3. "A join group request with member.name set will be
> > > > > > treated as
> > > > > > >     > >>>> `static-membership` strategy", in this case, how would
> > > the
> > > > > > switch from
> > > > > > >     > >>>> dynamic to static happen, since whoever changed the
> > > > > > member.name to
> > > > > > >     > >>>> not-null
> > > > > > >     > >>>> will be rejected, right?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 4. "just erase the cached mapping, and wait for
> > session
> > > > > > timeout to
> > > > > > >     > >>> trigger
> > > > > > >     > >>>> rebalance should be sufficient." this is also a bit
> > > > unclear
> > > > > > to me: who
> > > > > > >     > >>> will
> > > > > > >     > >>>> erase the cached mapping? Since it is on the
> > > broker-side I
> > > > > > assume that
> > > > > > >     > >>>> broker has to do it. Are you suggesting to use a new
> > > > request
> > > > > > for it?
> > > > > > >     > >>>>
> > > > > > >     > >>>> 5. "Halfway switch": following 3) above, if your
> > > proposal
> > > > is
> > > > > > basically
> > > > > > >     > >> to
> > > > > > >     > >>>> let "first join-request wins", and the strategy will
> > > stay
> > > > as
> > > > > > is until
> > > > > > >     > >> all
> > > > > > >     > >>>> members are gone, then this will also not happen since
> > > > > > whoever used
> > > > > > >     > >>>> different strategy as the first guy who sends
> > join-group
> > > > > > request will
> > > > > > >     > >> be
> > > > > > >     > >>>> rejected right?
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> Guozhang
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > > > > john@confluent.io>
> > > > > > >     > >> wrote:
> > > > > > >     > >>>>
> > > > > > >     > >>>>> This sounds good to me!
> > > > > > >     > >>>>>
> > > > > > >     > >>>>> Thanks for the time you've spent on it,
> > > > > > >     > >>>>> -John
> > > > > > >     > >>>>>
> > > > > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > > > > bchen11@outlook.com>
> > > > > > >     > >>>> wrote:
> > > > > > >     > >>>>>
> > > > > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy
> > > recently
> > > > > and
> > > > > > >     > >> haven't
> > > > > > >     > >>>> got
> > > > > > >     > >>>>>> time to update this thread. To summarize what we
> > come
> > > up
> > > > > so
> > > > > > far,
> > > > > > >     > >> here
> > > > > > >     > >>>> is
> > > > > > >     > >>>>> a
> > > > > > >     > >>>>>> draft updated plan:
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Introduce a new config called `member.name` which
> > is
> > > > > > supposed to
> > > > > > >     > >> be
> > > > > > >     > >>>>>> provided uniquely by the consumer client. The broker
> > > > will
> > > > > > maintain
> > > > > > >     > >> a
> > > > > > >     > >>>>> cache
> > > > > > >     > >>>>>> with [key:member.name, value:member.id]. A join
> > group
> > > > > > request with
> > > > > > >     > >>>>>> member.name set will be treated as
> > > `static-membership`
> > > > > > strategy,
> > > > > > >     > >> and
> > > > > > >     > >>>>> will
> > > > > > >     > >>>>>> reject any join group request without member.name.
> > So
> > > > > this
> > > > > > >     > >>>> coordination
> > > > > > >     > >>>>>> change will be differentiated from the
> > > > > `dynamic-membership`
> > > > > > >     > >> protocol
> > > > > > >     > >>> we
> > > > > > >     > >>>>>> currently have.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When handling static join group request:
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>  1.   The broker will check the membership to see
> > > > whether
> > > > > > this is
> > > > > > >     > >> a
> > > > > > >     > >>>> new
> > > > > > >     > >>>>>> member. If new, broker allocate a unique member id,
> > > > cache
> > > > > > the
> > > > > > >     > >> mapping
> > > > > > >     > >>>> and
> > > > > > >     > >>>>>> move to rebalance stage.
> > > > > > >     > >>>>>>  2.   Following 1, if this is an existing member,
> > > broker
> > > > > > will not
> > > > > > >     > >>>> change
> > > > > > >     > >>>>>> group state, and return its cached member.id and
> > > > current
> > > > > > >     > >> assignment.
> > > > > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > > > > >     > >>>>>>  3.   Although Guozhang has mentioned we could
> > rejoin
> > > > with
> > > > > > pair
> > > > > > >     > >>> member
> > > > > > >     > >>>>>> name and id, I think for join group request it is ok
> > > to
> > > > > > leave
> > > > > > >     > >> member
> > > > > > >     > >>> id
> > > > > > >     > >>>>>> blank as member name is the unique identifier. In
> > > commit
> > > > > > offset
> > > > > > >     > >>> request
> > > > > > >     > >>>>> we
> > > > > > >     > >>>>>> *must* have both.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When handling commit offset request, if enabled with
> > > > > static
> > > > > > >     > >>> membership,
> > > > > > >     > >>>>>> each time the commit request must have both
> > > member.name
> > > > > and
> > > > > > >     > >>> member.id
> > > > > > >     > >>>> to
> > > > > > >     > >>>>>> be identified as a `certificated member`. If not,
> > this
> > > > > > means there
> > > > > > >     > >>> are
> > > > > > >     > >>>>>> duplicate consumer members with same member name and
> > > the
> > > > > > request
> > > > > > >     > >> will
> > > > > > >     > >>>> be
> > > > > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> > > > client
> > > > > > will
> > > > > > >     > >> send a
> > > > > > >     > >>>>>> leave group request (static membership mode). In
> > > static
> > > > > > membership,
> > > > > > >     > >>> we
> > > > > > >     > >>>>> will
> > > > > > >     > >>>>>> also define `change-group-timeout` to hold on
> > > rebalance
> > > > > > provided by
> > > > > > >     > >>>>> leader.
> > > > > > >     > >>>>>> So we will wait for all the members to rejoin the
> > > group
> > > > > and
> > > > > > do
> > > > > > >     > >>> exactly
> > > > > > >     > >>>>> one
> > > > > > >     > >>>>>> rebalance since all members are expected to rejoin
> > > > within
> > > > > > timeout.
> > > > > > >     > >> If
> > > > > > >     > >>>>>> consumer crashes, the join group request from the
> > > > > restarted
> > > > > > >     > >> consumer
> > > > > > >     > >>>> will
> > > > > > >     > >>>>>> be recognized as an existing member and be handled
> > as
> > > > > above
> > > > > > >     > >> condition
> > > > > > >     > >>>> 1;
> > > > > > >     > >>>>>> However, if the consumer takes longer than session
> > > > timeout
> > > > > > to
> > > > > > >     > >> return,
> > > > > > >     > >>>> we
> > > > > > >     > >>>>>> shall still trigger rebalance but it could still try
> > > to
> > > > > > catch
> > > > > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > > > > timeout, its
> > > > > > >     > >>>> cached
> > > > > > >     > >>>>>> state on broker will be garbage collected and
> > trigger
> > > a
> > > > > new
> > > > > > >     > >> rebalance
> > > > > > >     > >>>>> when
> > > > > > >     > >>>>>> it finally joins.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> And consider the switch between dynamic to static
> > > > > > membership.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall
> > revise
> > > > the
> > > > > > >     > >> membership
> > > > > > >     > >>>> to
> > > > > > >     > >>>>>> static and wait for all the current members to
> > > restart,
> > > > > > since their
> > > > > > >     > >>>>>> membership is still dynamic. Here our assumption is
> > > that
> > > > > the
> > > > > > >     > >> restart
> > > > > > >     > >>>>>> process shouldn't take a long time, as long restart
> > is
> > > > > > breaking the
> > > > > > >     > >>>>>> `rebalance timeout` in whatever membership protocol
> > we
> > > > are
> > > > > > using.
> > > > > > >     > >>>> Before
> > > > > > >     > >>>>>> restart, all dynamic member join requests will be
> > > > > rejected.
> > > > > > >     > >>>>>>  2.  Static to dynamic: this is more like a
> > downgrade
> > > > > which
> > > > > > should
> > > > > > >     > >>> be
> > > > > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > > > > session
> > > > > > timeout
> > > > > > >     > >>> to
> > > > > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > > > > current
> > > > > > >     > >>> behavior)
> > > > > > >     > >>>>>>  3.  Halfway switch: a corner case is like some
> > > clients
> > > > > keep
> > > > > > >     > >> dynamic
> > > > > > >     > >>>>>> membership while some keep static membership. This
> > > will
> > > > > > cause the
> > > > > > >     > >>> group
> > > > > > >     > >>>>>> rebalance forever without progress because
> > > > dynamic/static
> > > > > > states
> > > > > > >     > >> are
> > > > > > >     > >>>>>> bouncing each other. This could guarantee that we
> > will
> > > > not
> > > > > > make the
> > > > > > >     > >>>>>> consumer group work in a wrong state by having half
> > > > static
> > > > > > and half
> > > > > > >     > >>>>> dynamic.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> To guarantee correctness, we will also push the
> > member
> > > > > > name/id pair
> > > > > > >     > >>> to
> > > > > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out)
> > and
> > > > > > upgrade the
> > > > > > >     > >> API
> > > > > > >     > >>>>>> version, these details will be further discussed
> > back
> > > in
> > > > > > the KIP.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Are there any concern for this high level proposal?
> > > Just
> > > > > > want to
> > > > > > >     > >>>>> reiterate
> > > > > > >     > >>>>>> on the core idea of the KIP: "If the broker
> > recognize
> > > > this
> > > > > > consumer
> > > > > > >     > >>> as
> > > > > > >     > >>>> an
> > > > > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Thanks a lot for everyone's input! I feel this
> > > proposal
> > > > is
> > > > > > much
> > > > > > >     > >> more
> > > > > > >     > >>>>>> robust than previous one!
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Best,
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Boyang
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> ________________________________
> > > > > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > > > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > > > > >     > >>>>>> To: dev@kafka.apache.org
> > > > > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple
> > > consumer
> > > > > > rebalances
> > > > > > >     > >>> by
> > > > > > >     > >>>>>> specifying member id
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Hi,
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> > > > about
> > > > > > internals
> > > > > > >     > >>>> again
> > > > > > >     > >>>>>> :)
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> I like the idea or a user config `member.name` and
> > to
> > > > > keep
> > > > > > `
> > > > > > >     > >>> member.id`
> > > > > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > > > > client.id`
> > > > > > might
> > > > > > >     > >>> not
> > > > > > >     > >>>>>> be a good idea.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> To clarify the algorithm, each time we generate a
> > new
> > > `
> > > > > > member.id`,
> > > > > > >     > >>> we
> > > > > > >     > >>>>>> also need to update the "group membership"
> > information
> > > > > (ie,
> > > > > > mapping
> > > > > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `
> > > member.id
> > > > `
> > > > > > replaces
> > > > > > >     > >>> the
> > > > > > >     > >>>>>> old entry in the cache.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> I also think, we need to preserve the `member.name
> > ->
> > > > > > member.id`
> > > > > > >     > >>>> mapping
> > > > > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should
> > > mention
> > > > > > this IMHO.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> For changing the default value of config
> > > > > > `leave.group.on.close`. I
> > > > > > >     > >>>> agree
> > > > > > >     > >>>>>> with John, that we should not change the default
> > > config,
> > > > > > because it
> > > > > > >     > >>>>>> would impact all consumer groups with dynamic
> > > > assignment.
> > > > > > However,
> > > > > > >     > >> I
> > > > > > >     > >>>>>> think we can document, that if static assignment is
> > > used
> > > > > > (ie,
> > > > > > >     > >>>>>> `member.name` is configured) we never send a
> > > > > > LeaveGroupRequest
> > > > > > >     > >>>>>> regardless of the config. Note, that the config is
> > > > > > internal, so not
> > > > > > >     > >>>> sure
> > > > > > >     > >>>>>> how to document this in detail. We should not expose
> > > the
> > > > > > internal
> > > > > > >     > >>>> config
> > > > > > >     > >>>>>> in the docs.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> About upgrading: why do we need have two rolling
> > > bounces
> > > > > > and encode
> > > > > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> If we upgrade an existing consumer group from
> > dynamic
> > > to
> > > > > > static, I
> > > > > > >     > >>>> don't
> > > > > > >     > >>>>>> see any reason why both should not work together and
> > > > > single
> > > > > > rolling
> > > > > > >     > >>>>>> bounce would not be sufficient? If we bounce the
> > first
> > > > > > consumer and
> > > > > > >     > >>>>>> switch from dynamic to static, it sends a `
> > > member.name`
> > > > > > and the
> > > > > > >     > >>> broker
> > > > > > >     > >>>>>> registers the [member.name, member.id] in the
> > cache.
> > > > Why
> > > > > > would
> > > > > > >     > >> this
> > > > > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > > > > assignment?
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> Also, Guozhang mentioned that for all other request,
> > > we
> > > > > > need to
> > > > > > >     > >> check
> > > > > > >     > >>>> if
> > > > > > >     > >>>>>> the mapping [member.name, member.id] contains the
> > > send
> > > > `
> > > > > > member.id`
> > > > > > >     > >>> --
> > > > > > >     > >>>> I
> > > > > > >     > >>>>>> don't think this is necessary -- it seems to be
> > > > sufficient
> > > > > > to check
> > > > > > >     > >>> the
> > > > > > >     > >>>>>> `member.id` from the [member.id, Assignment]
> > mapping
> > > as
> > > > > be
> > > > > > do
> > > > > > >     > >> today
> > > > > > >     > >>> --
> > > > > > >     > >>>>>> thus, checking `member.id` does not require any
> > > change
> > > > > > IMHO.
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>> -Matthias
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > > > > >     > >>>>>>> @James
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> What you described is true: the transition from
> > > dynamic
> > > > > to
> > > > > > static
> > > > > > >     > >>>>>>> memberships are not thought through yet. But I do
> > not
> > > > > > think it is
> > > > > > >     > >>> an
> > > > > > >     > >>>>>>> impossible problem: note that we indeed moved the
> > > > offset
> > > > > > commit
> > > > > > >     > >>> from
> > > > > > >     > >>>> ZK
> > > > > > >     > >>>>>> to
> > > > > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is
> > > to
> > > > > > first to
> > > > > > >     > >>>>>>> double-commits on both zk and coordinator, and then
> > > do
> > > > a
> > > > > > second
> > > > > > >     > >>> round
> > > > > > >     > >>>>> to
> > > > > > >     > >>>>>>> turn the zk off.
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > > > > >     > >>>> two-rolling-bounce
> > > > > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag
> > > to
> > > > > > "static"
> > > > > > >     > >>> while
> > > > > > >     > >>>>>> keep
> > > > > > >     > >>>>>>> the registry-id field empty still, in this case,
> > the
> > > > > > coordinator
> > > > > > >     > >>>> still
> > > > > > >     > >>>>>>> follows the logic of "dynamic", accepting the
> > request
> > > > > while
> > > > > > >     > >>> allowing
> > > > > > >     > >>>>> the
> > > > > > >     > >>>>>>> protocol to be set to "static"; after the first
> > > rolling
> > > > > > bounce,
> > > > > > >     > >> the
> > > > > > >     > >>>>> group
> > > > > > >     > >>>>>>> protocol is already "static", then a second rolling
> > > > > bounce
> > > > > > is
> > > > > > >     > >>>> triggered
> > > > > > >     > >>>>>> and
> > > > > > >     > >>>>>>> this time we set the registry-id.
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> Guozhang
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > > > > >     > >> wushujames@gmail.com>
> > > > > > >     > >>>>>> wrote:
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> > > > this:
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > > > > wangguoz@gmail.com
> > > > > > >     > >>>
> > > > > > >     > >>>>> wrote:
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with
> > additional
> > > > > > fields:
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> > > > membership
> > > > > > >     > >>> protocols.
> > > > > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > > > > pre-defined
> > > > > > >     > >>> member
> > > > > > >     > >>>>> id.
> > > > > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> > > > optional
> > > > > > >     > >>>>>>>>> "group-change-timeout" value.
> > > > > > >     > >>>>>>>>>
> > > > > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the
> > > two
> > > > > > protocols
> > > > > > >     > >>> for
> > > > > > >     > >>>>> all
> > > > > > >     > >>>>>>>>> group members: we accept the protocol on the
> > first
> > > > > joined
> > > > > > >     > >> member
> > > > > > >     > >>> of
> > > > > > >     > >>>>> the
> > > > > > >     > >>>>>>>>> group, and if later joining members indicate a
> > > > > different
> > > > > > >     > >>> membership
> > > > > > >     > >>>>>>>>> protocol, we reject it. If the
> > group-change-timeout
> > > > > > value was
> > > > > > >     > >>>>> different
> > > > > > >     > >>>>>>>> to
> > > > > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > > > > application that
> > > > > > >     > >>>> wants
> > > > > > >     > >>>>>> to
> > > > > > >     > >>>>>>>> switch to using static membership? Let's say there
> > > are
> > > > > 10
> > > > > > >     > >>> instances
> > > > > > >     > >>>> of
> > > > > > >     > >>>>>> it.
> > > > > > >     > >>>>>>>> As the instances go through a rolling restart,
> > they
> > > > will
> > > > > > switch
> > > > > > >     > >>> from
> > > > > > >     > >>>>>>>> dynamic membership (the default?) to static
> > > > membership.
> > > > > > As each
> > > > > > >     > >>> one
> > > > > > >     > >>>>>> leaves
> > > > > > >     > >>>>>>>> the group and restarts, they will be rejected from
> > > the
> > > > > > group
> > > > > > >     > >>>> (because
> > > > > > >     > >>>>>> the
> > > > > > >     > >>>>>>>> group is currently using dynamic membership). The
> > > > group
> > > > > > will
> > > > > > >     > >>> shrink
> > > > > > >     > >>>>> down
> > > > > > >     > >>>>>>>> until there is 1 node handling all the traffic.
> > > After
> > > > > > that one
> > > > > > >     > >>>>> restarts,
> > > > > > >     > >>>>>>>> the group will switch over to static membership.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> Is that right? That means that the transition plan
> > > > from
> > > > > > dynamic
> > > > > > >     > >> to
> > > > > > >     > >>>>>> static
> > > > > > >     > >>>>>>>> membership isn't very smooth.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> > > > This
> > > > > > reminds
> > > > > > >     > >> me
> > > > > > >     > >>>> of
> > > > > > >     > >>>>>> the
> > > > > > >     > >>>>>>>> transition plans that were discussed for moving
> > from
> > > > > > >     > >>> zookeeper-based
> > > > > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers.
> > That
> > > > was
> > > > > > also
> > > > > > >     > >>> hard,
> > > > > > >     > >>>>> and
> > > > > > >     > >>>>>>>> ultimately we decided not to build that.
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>> -James
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>>
> > > > > > >     > >>>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>>
> > > > > > >     > >>>> --
> > > > > > >     > >>>> -- Guozhang
> > > > > > >     > >>>>
> > > > > > >     > >>>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >>
> > > > > > >     > >> --
> > > > > > >     > >> -- Guozhang
> > > > > > >     > >>
> > > > > > >     >
> > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > >
> > > > > --
> > > > > -Regards,
> > > > > Mayuresh R. Gharat
> > > > > (862) 250-7125
> > > > >
> > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> >

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Jason Gustafson <ja...@confluent.io>.
>
> If we initialize a set of member names (I assume ids = names here) on
> broker through Admin API, the client needs to pick up this information
> simultaneously which I doubt if there is a generic way to achieve that? It
> would also make the scaling operations difficult if we need to define the
> member names every time we change the member set which is an extra
> operation burden. From my daily ops experience, dynamically generate member
> names on client side would be easier. Is there a good approach to address
> this issue?


Yeah, that's a good question. I'm hoping someone with more kubernetes
experience will jump in here. Basically my goal is to have an approach
which maps nicely to StatefulSets (
https://kubernetes.io/docs/tutorials/stateful-application/basic-stateful-set/).
The pods in a stateful set have an ordinal index, which sounds similar to
the static ids that I was describing. You can scale up and down a stateful
set, but you would need a plugin to grow and shrink the consumer group.
Sounds like it could work, but I'm not sure if it's the best way.

At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> critical streaming services abnormality. One of the burden was the night
> shift which requires the oncaller to quickly resolve the issue and get the
> streaming application back on track, however there is a chance of miss. My
> concern was that if we forfeit the timeout on static membership to trigger
> rebalance, missing some pages during midnight could be negatively
> impacting the system performance since we may realize that some partitions
> stop working for a couple of hours already until next morning. So
> registration timeout serves as the "last line of defense" to guarantee
> liveness if no human intervention jumps in.


Thanks, this is helpful background. I agree this is a risk in the approach
I've suggested. If we take a step back, I think there are two gaps in the
protocol for stateful applications:

1. We don't have a way to detect the same member across failures or
restarts. I think streams has some heuristic to try and handle the common
cases (such as rolling restarts), but the proposal here solves the problem
in a more robust way.

2. We don't have a way to know what the expected membership of the group
is. This leads us to try tricks like inserting delays into the rebalance
logic so that the group membership has time to stabilize before we make any
decisions. In your proposal, we have an expansion timeout, which is
basically the same thing as far as I can tell.

I think the first problem is the most important, but it would be nice if we
can solve the second problem as well. If we have a way to indicate the
expected group members, then the group can respond to a change much more
quickly. There would be no need to wait 5 minutes for all members to join
and it would be robust in the presence of failures. Ironically, static
membership in this case makes the group more dynamic ;).

That said, I can see how the registration timeout would be an attractive
safety net in some cases. Perhaps it would be good enough if we have a way
to pre-register group members administratively? Members can still be
expired due to inactivity and we would have a way to get around the
rebalance delays. Would that work?

Thanks,
Jason


On Wed, Nov 14, 2018 at 10:24 PM, Boyang Chen <bc...@outlook.com> wrote:

> Thank you for the clarification Jason! The proposals make sense here and
> let me continue the discussion.
>
> > Then the ids would be determined using some convention. Most likely, we
> would just use sequential numbers 0, 1, 2,
> > etc. We do the same thing for partition ids.
>
>
> If we initialize a set of member names (I assume ids = names here) on
> broker through Admin API, the client needs to pick up this information
> simultaneously which I doubt if there is a generic way to achieve that? It
> would also make the scaling operations difficult if we need to define the
> member names every time we change the member set which is an extra
> operation burden. From my daily ops experience, dynamically generate member
> names on client side would be easier. Is there a good approach to address
> this issue?
>
> > I was thinking that the registration is specified ahead of time and
> remains valid until changed. It would be more like a
> > replica assignment. We don't move partitions just because a broker is
> down.
> > The expectation is that it the broker will eventually return.
>
>
> At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the
> critical streaming services abnormality. One of the burden was the night
> shift which requires the oncaller to quickly resolve the issue and get the
> streaming application back on track, however there is a chance of miss. My
> concern was that if we forfeit the timeout on static membership to trigger
> rebalance, missing some pages during midnight could be negatively impacting
> the system performance since we may realize that some partitions stop
> working for a couple of hours already until next morning. So registration
> timeout serves as the "last line of defense" to guarantee liveness if no
> human intervention jumps in.
>
>
> I'm very interested in the replication protocol currently implemented on
> Kafka, but I'm not familiar with it. If we do have mechanism to handle
> issues like I mentioned above for replication (auto healing during
> mid-night if one broker is never back), we could continue discussing the
> new approaches to have basic guarantee of consumer group liveness.
>
>
> The discussion so far is to make sure that all the design approaches we
> have taken are pointing to real scenarios. Once we clarify the scenarios,
> we would definitely propose better solution on top of it. I hope these
> discussions make sense. Thanks again for helping make the design solid!
>
>
> Boyang
>
> ________________________________
> From: Jason Gustafson <ja...@confluent.io>
> Sent: Thursday, November 15, 2018 9:54 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> >
> > I feel this would make the current protocol harder to use. For example,
> on
> > KStream we typically would expect (number of instances * number of
> threads
> > per instance) consumers. Giving out all this many member names in a list
> > may not be easy, compared with dynamic generation of member names, at
> least
> > for KStream use case.
>
>
> That's a fair point. What I had in mind is for the API to specify the
> number of consumers in the group. Then the ids would be determined using
> some convention. Most likely, we would just use sequential numbers 0, 1, 2,
> etc. We do the same thing for partition ids. Streams folks can chime in and
> say whether that would work or not.
>
> So this suggests we will not rely on heartbeat and commit interval to
> > trigger rebalance? Even in static membership, I feel tracking active
> > members is still required to be handled by broker since not all users are
> > fully equipped with monitoring tools, otherwise users will feel
> > uncomfortable using static membership solely for reducing rebalance
> purpose.
>
>
> Let me clarify. I think it is still useful to distinguish the liveness of
> the consumer using the heartbeat mechanism and to propagate that
> information to the leader during rebalances. This gives the group leader
> the option (but not the requirement) to change the partition assignment for
> inactive members. My suggestion was more about how long the static
> registration would remain valid. In the current proposal we have a
> registration timeout, and I think Mayuresh is fair to point out the
> potential confusion with the session timeout. It's kind of the same, but
> not exactly. So instead, I was thinking that the registration is specified
> ahead of time and remains valid until changed. It would be more like a
> replica assignment. We don't move partitions just because a broker is down.
> The expectation is that it the broker will eventually return. Similarly for
> stateful applications, we would have the option to do the same thing. The
> registration is fixed until someone changes it.
>
> Does that make sense?
>
> Thanks,
> Jason
>
>
> On Wed, Nov 14, 2018 at 2:46 PM, Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Jason for the suggestions! I update the KIP with full schema
> > changes.
> >
> > > we offer an admin API that lets a user define the expected members of
> > the group.
> >
> > I feel this would make the current protocol harder to use. For example,
> on
> > KStream we typically would expect (number of instances * number of
> threads
> > per instance) consumers. Giving out all this many member names in a list
> > may not be easy, compared with dynamic generation of member names, at
> least
> > for KStream use case.
> >
> > > For the sake of discussion, I was wondering if we could just say that
> > static members do not expire.
> >
> >
> > So this suggests we will not rely on heartbeat and commit interval to
> > trigger rebalance? Even in static membership, I feel tracking active
> > members is still required to be handled by broker since not all users are
> > fully equipped with monitoring tools, otherwise users will feel
> > uncomfortable using static membership solely for reducing rebalance
> purpose.
> >
> >
> > For the admin API design, I'm simplifying the join group request
> handling,
> > while using admin tool to switch between static and dynamic membership
> and
> > set the two corresponding timeouts. Do you think this approach makes
> sense?
> > The version one implementation will be much more clean if we handle
> > membership change through user intervention.
> >
> >
> > Best,
> >
> > Boyang
> >
> > ________________________________
> > From: Jason Gustafson <ja...@confluent.io>
> > Sent: Wednesday, November 14, 2018 9:31 AM
> > To: dev
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hey Boyang,
> >
> > Thanks for the updates. From a high level, I think this actually
> > complements Konstantine's writeup on incremental rebalancing. The gap
> we're
> > addressing is providing a way to bind the the partition assignment of a
> > group to a set of user-provided ids so that we are not so reliant on the
> > group's immediate state. For example, these ids might identify the state
> > store volume for particular streams instances. This is basically what you
> > need to work well with k8s stateful sets (as far as I understand them).
> >
> > One key decision is how we would define and update the expected static
> > members in a consumer group. The mechanics of the registration and
> > expansion timeouts feel a little bit clunky. For the sake of discussion,
> I
> > was wondering if we could just say that static members do not expire.
> > Instead, we offer an admin API that lets a user define the expected
> members
> > of the group. This API could be used to both grow and shrink a group.
> This
> > would solve the rebalancing problems when applications are initially
> > bootstrapped or when they are restarted because we would always know how
> > many members should be in a group. What do you think?
> >
> > By the way, it would be helpful to include the full schema definition for
> > any protocol changes in the proposal.
> >
> > Thanks,
> > Jason
> >
> >
> > On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com>
> wrote:
> >
> > > Thanks Mayuresh for the feedback! Do you have a quick example for
> passing
> > > in consumer config dynamically? I mainly use Kafka Streams at my daily
> > work
> > > so probably missing the idea how to do it in the current consumer
> > setting.
> > >
> > >
> > > For differentiating session timeout and registration timeout, I would
> try
> > > to enhance the documentation in the first stage to see how people react
> > to
> > > the confusion (would be great if they feel straightforward!). Since one
> > > doesn't have to fully understand the difference unless defining the new
> > > config "member name", for current users we could buy some time to
> listen
> > to
> > > their understandings and improve our documentation correspondingly in
> the
> > > follow-up KIPs.
> > >
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Mayuresh Gharat <gh...@gmail.com>
> > > Sent: Sunday, November 11, 2018 1:06 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the reply.
> > >
> > > Please find the replies inline below :
> > > For having a consumer config at runtime, I think it's not necessary to
> > > address in this KIP because most companies run sidecar jobs through
> > daemon
> > > software like puppet. It should be easy to change the config through
> > script
> > > or UI without actual code change. We still want to leave flexibility
> for
> > > user to define member name as they like.
> > > ---- This might be little different for companies that use
> configuration
> > > management tools that does not allow the applications to define/change
> > the
> > > configs dynamically. For example, if we use something similar to spring
> > to
> > > pull in the configs for the KafkaConsumer and pass it to the
> constructor
> > to
> > > create the KafkaConsumer object, it will be hard to specify a unique
> > value
> > > to the "MEMBER_NAME" config unless someone deploying the app generates
> a
> > > unique string for this config outside the deployment workflow and
> copies
> > it
> > > statically before starting up each consumer instance. Unless we can
> > loosen
> > > the criteria for uniqueness of this config value, for each consumer
> > > instance in the consumer group, I am not sure of a better way of
> > > addressing this. If we don't want to loosen the criteria, then
> providing
> > a
> > > dynamic way to pass this in at runtime, would put the onus of having
> the
> > > same unique value each time a consumer is restarted, on to the
> > application
> > > that is running the consumer.
> > >
> > > I just updated the kip about having both "registration timeout" and
> > > "session timeout". The benefit of having two configs instead of one is
> to
> > > reduce the mental burden for operation, for example user just needs to
> > > unset "member name" to cast back to dynamic membership without worrying
> > > about tuning the "session timeout" back to a smaller value.
> > > --- That is a good point. I was thinking, if both the configs are
> > > specified, it would be confusing for the end user without understanding
> > the
> > > internals of the consumer and its interaction with group coordinator,
> as
> > > which takes precedence when and how it affects the consumer behavior.
> > Just
> > > my 2 cents.
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > > On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > > > Hey Mayuresh,
> > > >
> > > >
> > > > thanks for the thoughtful questions! Let me try to answer your
> > questions
> > > > one by one.
> > > >
> > > >
> > > > For having a consumer config at runtime, I think it's not necessary
> to
> > > > address in this KIP because most companies run sidecar jobs through
> > > daemon
> > > > software like puppet. It should be easy to change the config through
> > > script
> > > > or UI without actual code change. We still want to leave flexibility
> > for
> > > > user to define member name as they like.
> > > >
> > > >
> > > > I just updated the kip about having both "registration timeout" and
> > > > "session timeout". The benefit of having two configs instead of one
> is
> > to
> > > > reduce the mental burden for operation, for example user just needs
> to
> > > > unset "member name" to cast back to dynamic membership without
> worrying
> > > > about tuning the "session timeout" back to a smaller value.
> > > >
> > > >
> > > > For backup topic, I think it's a low-level detail which could be
> > > addressed
> > > > in the implementation. I feel no preference of adding a new topic vs
> > > reuse
> > > > consumer offsets topic. I will do more analysis and make a trade-off
> > > > comparison. Nice catch!
> > > >
> > > >
> > > > I hope the explanations make sense to you. I will keep polishing on
> the
> > > > edge cases and details.
> > > >
> > > >
> > > > Best,
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Mayuresh Gharat <gh...@gmail.com>
> > > > Sent: Saturday, November 10, 2018 10:25 AM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Hi Boyang,
> > > >
> > > > Thanks for the KIP and sorry for being late to the party. This KIP is
> > > > really useful for us at Linkedin.
> > > >
> > > > I had a few questions :
> > > >
> > > > The idea of having static member name seems nice, but instead of a
> > > config,
> > > > would it be possible for it to be passed in to the consumer at
> runtime?
> > > > This is because an app might want to decide the config value at
> runtime
> > > > using its host information for example, to generate the unique member
> > > name.
> > > >
> > > > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > > > wondering if we can reuse the session timeout here. This might help
> us
> > to
> > > > have one less config on the consumer.
> > > >
> > > > The KIP also talks about adding another internal topic
> > > "static_member_map".
> > > > Would the semantics (GroupCoordinator broker, topic configs) be the
> > same
> > > as
> > > > __consumer_offsets topic?
> > > >
> > > > Thanks,
> > > >
> > > > Mayuresh
> > > >
> > > >
> > > > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com>
> > wrote:
> > > >
> > > > > I took a quick pass of the proposal. First I would say it's a very
> > > > > brilliant initiative from Konstantine and Confluent folks. To draft
> > up
> > > a
> > > > > proposal like this needs deep understanding of the rebalance
> > protocol!
> > > I
> > > > > summarized some thoughts here.
> > > > >
> > > > >
> > > > > Overall the motivations of the two proposals align on that:
> > > > >
> > > > >   1.  Both believe the invariant resource (belonging to the same
> > > process)
> > > > > should be preserved across rebalance.
> > > > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > > > redistribution. I don't use rebalance here since part one of the
> > > > > cooperative proposal could potentially introduce more rebalances
> but
> > > only
> > > > > on must-move resources.
> > > > >   3.  Scale up/down and rolling bounce are causing unnecessary
> > resource
> > > > > shuffling that need to be mitigated.
> > > > >
> > > > >
> > > > > On motivation level, I think both approach could solve/mitigate the
> > > above
> > > > > issues. They are just different in design philosophy, or I would
> say
> > > the
> > > > > perspective difference between framework user and algorithm
> designer.
> > > > >
> > > > >
> > > > > Two proposals have different focuses. KIP-345 is trying to place
> more
> > > > > fine-grained control on the broker side to reduce the unnecessary
> > > > > rebalances, while keeping the client logic intact. This is pretty
> > > > intuitive
> > > > > cause-effect for normal developers who are not very familiar with
> > > > rebalance
> > > > > protocol. As a developer working with Kafka Streams daily, I'd be
> > happy
> > > > to
> > > > > see a simplified rebalance protocol and just focus on maintaining
> the
> > > > > stream/consumer jobs. Too many rebalances raised my concern on the
> > job
> > > > > health. To be concise, static membership has the advantage of
> > reducing
> > > > > mental burden.
> > > > >
> > > > >
> > > > > Cooperative proposal takes thoughtful approach on client side. We
> > want
> > > to
> > > > > have fine-grained control on the join/exit group behaviors and make
> > the
> > > > > current dynamic membership better to address above issues. I do
> feel
> > > our
> > > > > idea crossed on the delayed rebalance when we scale up/down, which
> > > could
> > > > > potentially reduce the state shuffling and decouple the behavior
> from
> > > > > session timeout which is already overloaded.  In this sense, I
> > believe
> > > > both
> > > > > approaches would serve well in making "reasonable rebalance" happen
> > at
> > > > the
> > > > > "right timing".
> > > > >
> > > > >
> > > > > However, based on my understanding, either 345 or cooperative
> > > rebalancing
> > > > > is not solving the problem Mike has proposed: could we do a better
> > job
> > > at
> > > > > scaling up/down in ideal timing? My initial response was to
> introduce
> > > an
> > > > > admin API which now I feel is sub-optimal, in that the goal of
> smooth
> > > > > transition is to make sure the newly up hosts are actually "ready".
> > For
> > > > > example:
> > > > >
> > > > >
> > > > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At
> > some
> > > > > time we would like to scale up to 8 hosts, with the current
> > > improvements
> > > > we
> > > > > could reduce 4 potential rebalances to a single one. But the new
> > hosts
> > > > are
> > > > > yet unknown to be "ready" if they need to reconstruct the local
> > state.
> > > To
> > > > > be actually ready, we need 4 standby tasks running on those empty
> > hosts
> > > > and
> > > > > leader needs to wait for the signal of "replay/reconstruct
> complete"
> > to
> > > > > actually involve them into the main consumer group. Otherwise,
> > > rebalance
> > > > > just kills our performance since we need to wait indefinite long
> for
> > > task
> > > > > migration.
> > > > >
> > > > >
> > > > > The scale down is also tricky such that we are not able to define a
> > > > "true"
> > > > > leave of a member. Rebalance immediately after "true" leaves are
> most
> > > > > optimal comparing with human intervention. Does this make sense?
> > > > >
> > > > >
> > > > > My intuition is that cooperative approach which was implemented on
> > the
> > > > > client side could better handle scaling cases than KIP 345, since
> it
> > > > > involves a lot of algorithmic changes to define "replaying" stage,
> > > which
> > > > I
> > > > > feel would over-complicate broker logic if implemented on
> > coordinator.
> > > If
> > > > > we let 345 focus on reducing unnecessary rebalance, and let
> > cooperative
> > > > > approach focus on judging best timing of scale up/down, the two
> > efforts
> > > > > could be aligned. In long term, I feel the more complex improvement
> > of
> > > > > consumer protocol should happen on client side instead of server
> side
> > > > which
> > > > > is easier to test and has less global impact for the entire Kafka
> > > > > production cluster.
> > > > >
> > > > >
> > > > > Thanks again to Konstantine, Matthias and other folks in coming up
> > with
> > > > > this great client proposal. This is great complementation to KIP
> 345.
> > > In
> > > > a
> > > > > high level, we are not having any collision on the path and both
> > > > proposals
> > > > > are making sense here. Just need better sync to avoid duplicate
> > effort
> > > :)
> > > > >
> > > > >
> > > > > Best,
> > > > >
> > > > > Boyang
> > > > >
> > > > >
> > > > > ________________________________
> > > > > From: Boyang Chen <bc...@outlook.com>
> > > > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Thanks Matthias for bringing this awesome proposal up! I shall
> take a
> > > > > deeper look and make a comparison between the two proposals.
> > > > >
> > > > >
> > > > > Meanwhile for the scale down specifically for stateful streaming,
> we
> > > > could
> > > > > actually introduce a new status called "learner" where the newly up
> > > hosts
> > > > > could try to catch up with the assigned task progress first before
> > > > > triggering the rebalance, from which we don't see a sudden dip on
> the
> > > > > progress. However, it is built on top of the success of KIP-345.
> > > > >
> > > > >
> > > > > ________________________________
> > > > > From: Matthias J. Sax <ma...@confluent.io>
> > > > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > > specifying member id
> > > > >
> > > > > Hey,
> > > > >
> > > > > there was quite a pause on this KIP discussion and in the mean
> time,
> > a
> > > > > new design for incremental cooporative rebalance was suggested:
> > > > >
> > > > >
> > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > https://cwiki.apache.org/confluence/display/KAFKA/Increm
> > > ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> > > cwiki.apache.org
> > > Rebalancing between distributed application processes in Apache Kafka
> was
> > > enhanced considerably when it was decoupled as logic from Kafka brokers
> > and
> > > was moved as responsibility to the clients and specifically to Kafka
> > > Consumer. This pattern has been working robustly for quite a while now
> > and
> > > has ...
> > >
> > >
> > >
> > > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > > >
> > > > cwiki.apache.org
> > > > Rebalancing between distributed application processes in Apache Kafka
> > was
> > > > enhanced considerably when it was decoupled as logic from Kafka
> brokers
> > > and
> > > > was moved as responsibility to the clients and specifically to Kafka
> > > > Consumer. This pattern has been working robustly for quite a while
> now
> > > and
> > > > has ...
> > > >
> > > >
> > > >
> > > > >
> > > > >
> > > > > We should make sure that the proposal and this KIP align to each
> > other.
> > > > > Thoughts?
> > > > >
> > > > >
> > > > > -Matthias
> > > > >
> > > > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > > > Hey Mike,
> > > > > >
> > > > > >
> > > > > > thanks for the feedback, the two question are very thoughtful!
> > > > > >
> > > > > >
> > > > > >> 1) I am a little confused about the distinction for the leader.
> If
> > > the
> > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > quickly
> > > > > comes up) to update application code, will a rebalance be
> triggered?
> > I
> > > >
> > > > do
> > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > >
> > > > > > For Q1 my intention was to minimize the change within one KIP,
> > since
> > > > the
> > > > > leader rejoining case could be addressed separately.
> > > > > >
> > > > > >
> > > > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > gracefully increase the number of nodes in the cluster. I think we
> > need
> > > > to
> > > > > support graceful shrink down as well. If I set the registration
> > timeout
> > > > to
> > > > > 5 minutes > to handle rolling restarts or intermittent failures
> > without
> > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > group
> > > to
> > > > > rebalance if I am intentionally removing a node from the cluster. I
> > am
> > > > not
> > > > > sure the best way to > do this. One idea I had was adding the
> ability
> > > > for a
> > > > > CLI or Admin API to force a rebalance of the group. This would
> allow
> > > for
> > > > an
> > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > registration timeout on > shrink down. What do you think?
> > > > > >
> > > > > > For 2) my understanding is that for scaling down case it is
> better
> > to
> > > > be
> > > > > addressed by CLI tool than code logic, since only by human
> evaluation
> > > we
> > > > > could decide whether it is a "right timing" -- the time when all
> the
> > > > > scaling down consumers are offline -- to kick in rebalance. Unless
> we
> > > > > introduce another term on coordinator which indicates the target
> > > consumer
> > > > > group size, broker will find it hard to decide when to start
> > rebalance.
> > > > So
> > > > > far I prefer to hold the implementation for that, but agree we
> could
> > > > > discuss whether we want to introduce admin API in this KIP or a
> > > separate
> > > > > one.
> > > > > >
> > > > > >
> > > > > > Thanks again for the proposed ideas!
> > > > > >
> > > > > >
> > > > > > Boyang
> > > > > >
> > > > > > ________________________________
> > > > > > From: Mike Freyberger <mi...@xandr.com>
> > > > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > > > To: dev@kafka.apache.org
> > > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > > by
> > > > > specifying member id
> > > > > >
> > > > > > Boyang,
> > > > > >
> > > > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > > > >
> > > > > > 1) I am a little confused about the distinction for the leader.
> If
> > > the
> > > > > consumer node that was assigned leader does a bounce (goes down and
> > > > quickly
> > > > > comes up) to update application code, will a rebalance be
> triggered?
> > I
> > > do
> > > > > not think a bounce of the leader should trigger a rebalance.
> > > > > >
> > > > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > > > gracefully increase the number of nodes in the cluster. I think we
> > need
> > > > to
> > > > > support graceful shrink down as well. If I set the registration
> > timeout
> > > > to
> > > > > 5 minutes to handle rolling restarts or intermittent failures
> without
> > > > > shuffling state, I don't want to wait 5 minutes in order for the
> > group
> > > to
> > > > > rebalance if I am intentionally removing a node from the cluster. I
> > am
> > > > not
> > > > > sure the best way to do this. One idea I had was adding the ability
> > > for a
> > > > > CLI or Admin API to force a rebalance of the group. This would
> allow
> > > for
> > > > an
> > > > > admin to trigger the rebalance manually without waiting the entire
> > > > > registration timeout on shrink down. What do you think?
> > > > > >
> > > > > > Mike
> > > > > >
> > > > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com>
> wrote:
> > > > > >
> > > > > >     Btw, I updated KIP 345 based on my understanding. Feel free
> to
> > > take
> > > > > another round of look:
> > > > > >
> > > > > >
> > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > > KIP-345: Introduce static membership protocol to reduce ...<
> > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > >
> > > > > > cwiki.apache.org
> > > > > > For stateful applications, one of the biggest performance
> > bottleneck
> > > is
> > > > > the state shuffling. In Kafka consumer, there is a concept called
> > > > > "rebalance" which means that for given M partitions and N consumers
> > in
> > > > one
> > > > > consumer group, Kafka will try to balance the load between
> consumers
> > > and
> > > > > ideally have ...
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > > > >
> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > >
> > > > > >     cwiki.apache.org
> > > > > >     For stateful applications, one of the biggest performance
> > > > bottleneck
> > > > > is the state shuffling. In Kafka consumer, there is a concept
> called
> > > > > "rebalance" which means that for given M partitions and N consumers
> > in
> > > > one
> > > > > consumer group, Kafka will try to balance the load between
> consumers
> > > and
> > > > > ideally have ...
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >     ________________________________
> > > > > >     From: Boyang Chen <bc...@outlook.com>
> > > > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > > > >     To: dev@kafka.apache.org
> > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > > by specifying member id
> > > > > >
> > > > > >     Thanks everyone for the input on this thread! (Sorry it's
> been
> > a
> > > > > while) I feel that we are very close to the final solution.
> > > > > >
> > > > > >
> > > > > >     Hey Jason and Mike, I have two quick questions on the new
> > > features
> > > > > here:
> > > > > >
> > > > > >       1.  so our proposal is that until we add a new static
> member
> > > into
> > > > > the group (scale up), we will not trigger rebalance until the
> > > > "registration
> > > > > timeout"( the member has been offline for too long)? How about
> > leader's
> > > > > rejoin request, I think we should still trigger rebalance when that
> > > > > happens, since the consumer group may have new topics to consume?
> > > > > >       2.  I'm not very clear on the scale up scenario in static
> > > > > membership here. Should we fallback to dynamic membership while
> > > > > adding/removing hosts (by setting member.name = null), or we still
> > > want
> > > > > to add instances with `member.name` so that we eventually
> > > expand/shrink
> > > > > the static membership? I personally feel the easier solution is to
> > spin
> > > > up
> > > > > new members and wait until either the same "registration timeout"
> or
> > a
> > > > > "scale up timeout" before starting the rebalance. What do you
> think?
> > > > > >
> > > > > >     Meanwhile I will go ahead to make changes to the KIP with our
> > > newly
> > > > > discussed items and details. Really excited to see the design has
> > > become
> > > > > more solid.
> > > > > >
> > > > > >     Best,
> > > > > >     Boyang
> > > > > >
> > > > > >     ________________________________
> > > > > >     From: Jason Gustafson <ja...@confluent.io>
> > > > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > > > >     To: dev
> > > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > > > by specifying member id
> > > > > >
> > > > > >     Hey Mike,
> > > > > >
> > > > > >     Yeah, that's a good point. A long "registration timeout" may
> > not
> > > be
> > > > > a great
> > > > > >     idea. Perhaps in practice you'd set it long enough to be able
> > to
> > > > > detect a
> > > > > >     failure and provision a new instance. Maybe on the order of
> 10
> > > > > minutes is
> > > > > >     more reasonable.
> > > > > >
> > > > > >     In any case, it's probably a good idea to have an
> > administrative
> > > > way
> > > > > to
> > > > > >     force deregistration. One option is to extend the
> DeleteGroups
> > > API
> > > > > with a
> > > > > >     list of members names.
> > > > > >
> > > > > >     -Jason
> > > > > >
> > > > > >
> > > > > >
> > > > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > > > mfreyberger@appnexus.com>
> > > > > >     wrote:
> > > > > >
> > > > > >     > Jason,
> > > > > >     >
> > > > > >     > Regarding step 4 in your proposal which suggests beginning
> a
> > > long
> > > > > timer
> > > > > >     > (30 minutes) when a static member leaves the group, would
> > there
> > > > > also be the
> > > > > >     > ability for an admin to force a static membership
> expiration?
> > > > > >     >
> > > > > >     > I'm thinking that during particular types of outages or
> > > upgrades
> > > > > users
> > > > > >     > would want forcefully remove a static member from the
> group.
> > > > > >     >
> > > > > >     > So the user would shut the consumer down normally, which
> > > wouldn't
> > > > > trigger
> > > > > >     > a rebalance. Then the user could use an admin CLI tool to
> > force
> > > > > remove that
> > > > > >     > consumer from the group, so the TopicPartitions that were
> > > > > previously owned
> > > > > >     > by that consumer can be released.
> > > > > >     >
> > > > > >     > At a high level, we need consumer groups to gracefully
> handle
> > > > > intermittent
> > > > > >     > failures and permanent failures. Currently, the consumer
> > group
> > > > > protocol
> > > > > >     > handles permanent failures well, but does not handle
> > > intermittent
> > > > > failures
> > > > > >     > well (it creates unnecessary rebalances). I want to make
> sure
> > > the
> > > > > overall
> > > > > >     > solution here handles both intermittent failures and
> > permanent
> > > > > failures,
> > > > > >     > rather than sacrificing support for permanent failures in
> > order
> > > > to
> > > > > provide
> > > > > >     > support for intermittent failures.
> > > > > >     >
> > > > > >     > Mike
> > > > > >     >
> > > > > >     > Sent from my iPhone
> > > > > >     >
> > > > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > > > jason@confluent.io>
> > > > > wrote:
> > > > > >     > >
> > > > > >     > > Hey Guozhang,
> > > > > >     > >
> > > > > >     > > Responses below:
> > > > > >     > >
> > > > > >     > > Originally I was trying to kill more birds with one stone
> > > with
> > > > > KIP-345,
> > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > shutting
> > > > > down a
> > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> early
> > > > > email), and
> > > > > >     > >> hence proposing to have a pure static-membership
> protocol.
> > > But
> > > > > thinking
> > > > > >     > >> twice about it I now feel it may be too ambitious and
> > worth
> > > > > fixing in
> > > > > >     > >> another KIP.
> > > > > >     > >
> > > > > >     > >
> > > > > >     > > I was considering an extension to support
> > pre-initialization
> > > of
> > > > > the
> > > > > >     > static
> > > > > >     > > members of the group, but I agree we should probably
> leave
> > > this
> > > > > problem
> > > > > >     > for
> > > > > >     > > future work.
> > > > > >     > >
> > > > > >     > > 1. How this longish static member expiration timeout
> > defined?
> > > > Is
> > > > > it via a
> > > > > >     > >> broker, hence global config, or via a client config
> which
> > > can
> > > > be
> > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > >     > >
> > > > > >     > >
> > > > > >     > > I am not too sure. I tend to lean toward server-side
> > configs
> > > > > because they
> > > > > >     > > are easier to evolve. If we have to add something to the
> > > > > protocol, then
> > > > > >     > > we'll be stuck with it forever.
> > > > > >     > >
> > > > > >     > > 2. Assuming that for static members, LEAVE_GROUP request
> > will
> > > > not
> > > > > >     > trigger a
> > > > > >     > >> rebalance immediately either, similar to session
> timeout,
> > > but
> > > > > only the
> > > > > >     > >> longer member expiration timeout, can we remove the
> > > internal "
> > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > walk-around
> > > > > >     > then?
> > > > > >     > >
> > > > > >     > >
> > > > > >     > > Yeah, I hope we can ultimately get rid of it, but we may
> > need
> > > > it
> > > > > for
> > > > > >     > > compatibility with older brokers. A related question is
> > what
> > > > > should be
> > > > > >     > the
> > > > > >     > > behavior of the consumer if `member.name` is provided
> but
> > > the
> > > > > broker
> > > > > >     > does
> > > > > >     > > not support it? We could either fail or silently
> downgrade
> > to
> > > > > dynamic
> > > > > >     > > membership.
> > > > > >     > >
> > > > > >     > > -Jason
> > > > > >     > >
> > > > > >     > >
> > > > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > >     > wrote:
> > > > > >     > >>
> > > > > >     > >> Hey Jason,
> > > > > >     > >>
> > > > > >     > >> I like your idea to simplify the upgrade protocol to
> allow
> > > > > co-exist of
> > > > > >     > >> static and dynamic members. Admittedly it may make the
> > > > > coordinator-side
> > > > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > > > >     > >>
> > > > > >     > >> Originally I was trying to kill more birds with one
> stone
> > > with
> > > > > KIP-345,
> > > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > > shutting
> > > > > down a
> > > > > >     > >> multi-instance client (mentioned as case 1)/2) in my
> early
> > > > > email), and
> > > > > >     > >> hence proposing to have a pure static-membership
> protocol.
> > > But
> > > > > thinking
> > > > > >     > >> twice about it I now feel it may be too ambitious and
> > worth
> > > > > fixing in
> > > > > >     > >> another KIP. With that, I think what you've proposed
> here
> > > is a
> > > > > good way
> > > > > >     > to
> > > > > >     > >> go for KIP-345 itself.
> > > > > >     > >>
> > > > > >     > >> Note there are a few details in your proposal we'd still
> > > need
> > > > > to figure
> > > > > >     > >> out:
> > > > > >     > >>
> > > > > >     > >> 1. How this longish static member expiration timeout
> > > defined?
> > > > > Is it via
> > > > > >     > a
> > > > > >     > >> broker, hence global config, or via a client config
> which
> > > can
> > > > be
> > > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > > >     > >>
> > > > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> > > will
> > > > > not
> > > > > >     > trigger a
> > > > > >     > >> rebalance immediately either, similar to session
> timeout,
> > > but
> > > > > only the
> > > > > >     > >> longer member expiration timeout, can we remove the
> > > internal "
> > > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > > walk-around
> > > > > >     > then?
> > > > > >     > >>
> > > > > >     > >>
> > > > > >     > >>
> > > > > >     > >> Guozhang
> > > > > >     > >>
> > > > > >     > >>
> > > > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > > > jason@confluent.io>
> > > > > >     > >> wrote:
> > > > > >     > >>
> > > > > >     > >>> Hey All,
> > > > > >     > >>>
> > > > > >     > >>> Nice to see some solid progress on this. It sounds like
> > one
> > > > of
> > > > > the
> > > > > >     > >>> complications is allowing static and dynamic
> registration
> > > to
> > > > > coexist.
> > > > > >     > I'm
> > > > > >     > >>> wondering if we can do something like the following:
> > > > > >     > >>>
> > > > > >     > >>> 1. Statically registered members (those joining the
> group
> > > > with
> > > > > a
> > > > > >     > >> non-null `
> > > > > >     > >>> member.name`) maintain a session with the coordinator
> > just
> > > > > like
> > > > > >     > dynamic
> > > > > >     > >>> members.
> > > > > >     > >>> 2. If a session is active for a static member when a
> > > > rebalance
> > > > > begins,
> > > > > >     > >> then
> > > > > >     > >>> basically we'll keep the current behavior. The
> rebalance
> > > will
> > > > > await the
> > > > > >     > >>> static member joining the group.
> > > > > >     > >>> 3. If a static member does not have an active session,
> > then
> > > > the
> > > > > >     > >> coordinator
> > > > > >     > >>> will not wait for it to join, but will still include it
> > in
> > > > the
> > > > > >     > rebalance.
> > > > > >     > >>> The coordinator will forward the cached subscription
> > > > > information to the
> > > > > >     > >>> leader and will cache the assignment after the
> rebalance
> > > > > completes.
> > > > > >     > (Note
> > > > > >     > >>> that we still have the generationId to fence offset
> > commits
> > > > > from a
> > > > > >     > static
> > > > > >     > >>> zombie if the assignment changes.)
> > > > > >     > >>> 4. When a static member leaves the group or has its
> > session
> > > > > expire, no
> > > > > >     > >>> rebalance is triggered. Instead, we can begin a timer
> to
> > > > > expire the
> > > > > >     > >> static
> > > > > >     > >>> registration. This would be a longish timeout (like 30
> > > > minutes
> > > > > say).
> > > > > >     > >>>
> > > > > >     > >>> So basically static members participate in all
> rebalances
> > > > > regardless
> > > > > >     > >>> whether they have an active session. In a given
> > rebalance,
> > > > > some of the
> > > > > >     > >>> members may be static and some dynamic. The group
> leader
> > > can
> > > > > >     > >> differentiate
> > > > > >     > >>> the two based on the presence of the `member.name` (we
> > > have
> > > > > to add
> > > > > >     > this
> > > > > >     > >> to
> > > > > >     > >>> the JoinGroupResponse). Generally speaking, we would
> > choose
> > > > > leaders
> > > > > >     > >>> preferentially from the active members that support the
> > > > latest
> > > > > >     > JoinGroup
> > > > > >     > >>> protocol and are using static membership. If we have to
> > > > choose
> > > > > a leader
> > > > > >     > >>> with an old version, however, it would see all members
> in
> > > the
> > > > > group
> > > > > >     > >> (static
> > > > > >     > >>> or dynamic) as dynamic members and perform the
> assignment
> > > as
> > > > > usual.
> > > > > >     > >>>
> > > > > >     > >>> Would that work?
> > > > > >     > >>>
> > > > > >     > >>> -Jason
> > > > > >     > >>>
> > > > > >     > >>>
> > > > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com>
> > > > > >     > >> wrote:
> > > > > >     > >>>
> > > > > >     > >>>> Hello Boyang,
> > > > > >     > >>>>
> > > > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > > > >     > >>>>
> > > > > >     > >>>> 1. Where will "change-group-timeout" be communicated
> to
> > > the
> > > > > broker?
> > > > > >     > >> Will
> > > > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> > > going
> > > > > to
> > > > > >     > >>> piggy-back
> > > > > >     > >>>> on the existing session-timeout field (assuming that
> the
> > > > > original
> > > > > >     > value
> > > > > >     > >>>> will not be used anywhere in the static membership any
> > > > more)?
> > > > > >     > >>>>
> > > > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > > > timeout to
> > > > > >     > >> return,
> > > > > >     > >>>> we shall still trigger rebalance but it could still
> try
> > to
> > > > > catch
> > > > > >     > >>>> `change-group-timeout`.": what does this mean? I
> thought
> > > > your
> > > > > proposal
> > > > > >     > >> is
> > > > > >     > >>>> that for static memberships, the broker will NOT
> trigger
> > > > > rebalance
> > > > > >     > even
> > > > > >     > >>>> after session-timeout has been detected, but only that
> > > after
> > > > > >     > >>>> change-group-timeout
> > > > > >     > >>>> which is supposed to be longer than session-timeout to
> > be
> > > > > defined?
> > > > > >     > >>>>
> > > > > >     > >>>> 3. "A join group request with member.name set will be
> > > > > treated as
> > > > > >     > >>>> `static-membership` strategy", in this case, how would
> > the
> > > > > switch from
> > > > > >     > >>>> dynamic to static happen, since whoever changed the
> > > > > member.name to
> > > > > >     > >>>> not-null
> > > > > >     > >>>> will be rejected, right?
> > > > > >     > >>>>
> > > > > >     > >>>> 4. "just erase the cached mapping, and wait for
> session
> > > > > timeout to
> > > > > >     > >>> trigger
> > > > > >     > >>>> rebalance should be sufficient." this is also a bit
> > > unclear
> > > > > to me: who
> > > > > >     > >>> will
> > > > > >     > >>>> erase the cached mapping? Since it is on the
> > broker-side I
> > > > > assume that
> > > > > >     > >>>> broker has to do it. Are you suggesting to use a new
> > > request
> > > > > for it?
> > > > > >     > >>>>
> > > > > >     > >>>> 5. "Halfway switch": following 3) above, if your
> > proposal
> > > is
> > > > > basically
> > > > > >     > >> to
> > > > > >     > >>>> let "first join-request wins", and the strategy will
> > stay
> > > as
> > > > > is until
> > > > > >     > >> all
> > > > > >     > >>>> members are gone, then this will also not happen since
> > > > > whoever used
> > > > > >     > >>>> different strategy as the first guy who sends
> join-group
> > > > > request will
> > > > > >     > >> be
> > > > > >     > >>>> rejected right?
> > > > > >     > >>>>
> > > > > >     > >>>>
> > > > > >     > >>>> Guozhang
> > > > > >     > >>>>
> > > > > >     > >>>>
> > > > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > > > john@confluent.io>
> > > > > >     > >> wrote:
> > > > > >     > >>>>
> > > > > >     > >>>>> This sounds good to me!
> > > > > >     > >>>>>
> > > > > >     > >>>>> Thanks for the time you've spent on it,
> > > > > >     > >>>>> -John
> > > > > >     > >>>>>
> > > > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > > > bchen11@outlook.com>
> > > > > >     > >>>> wrote:
> > > > > >     > >>>>>
> > > > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy
> > recently
> > > > and
> > > > > >     > >> haven't
> > > > > >     > >>>> got
> > > > > >     > >>>>>> time to update this thread. To summarize what we
> come
> > up
> > > > so
> > > > > far,
> > > > > >     > >> here
> > > > > >     > >>>> is
> > > > > >     > >>>>> a
> > > > > >     > >>>>>> draft updated plan:
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Introduce a new config called `member.name` which
> is
> > > > > supposed to
> > > > > >     > >> be
> > > > > >     > >>>>>> provided uniquely by the consumer client. The broker
> > > will
> > > > > maintain
> > > > > >     > >> a
> > > > > >     > >>>>> cache
> > > > > >     > >>>>>> with [key:member.name, value:member.id]. A join
> group
> > > > > request with
> > > > > >     > >>>>>> member.name set will be treated as
> > `static-membership`
> > > > > strategy,
> > > > > >     > >> and
> > > > > >     > >>>>> will
> > > > > >     > >>>>>> reject any join group request without member.name.
> So
> > > > this
> > > > > >     > >>>> coordination
> > > > > >     > >>>>>> change will be differentiated from the
> > > > `dynamic-membership`
> > > > > >     > >> protocol
> > > > > >     > >>> we
> > > > > >     > >>>>>> currently have.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> When handling static join group request:
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>  1.   The broker will check the membership to see
> > > whether
> > > > > this is
> > > > > >     > >> a
> > > > > >     > >>>> new
> > > > > >     > >>>>>> member. If new, broker allocate a unique member id,
> > > cache
> > > > > the
> > > > > >     > >> mapping
> > > > > >     > >>>> and
> > > > > >     > >>>>>> move to rebalance stage.
> > > > > >     > >>>>>>  2.   Following 1, if this is an existing member,
> > broker
> > > > > will not
> > > > > >     > >>>> change
> > > > > >     > >>>>>> group state, and return its cached member.id and
> > > current
> > > > > >     > >> assignment.
> > > > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > > > >     > >>>>>>  3.   Although Guozhang has mentioned we could
> rejoin
> > > with
> > > > > pair
> > > > > >     > >>> member
> > > > > >     > >>>>>> name and id, I think for join group request it is ok
> > to
> > > > > leave
> > > > > >     > >> member
> > > > > >     > >>> id
> > > > > >     > >>>>>> blank as member name is the unique identifier. In
> > commit
> > > > > offset
> > > > > >     > >>> request
> > > > > >     > >>>>> we
> > > > > >     > >>>>>> *must* have both.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> When handling commit offset request, if enabled with
> > > > static
> > > > > >     > >>> membership,
> > > > > >     > >>>>>> each time the commit request must have both
> > member.name
> > > > and
> > > > > >     > >>> member.id
> > > > > >     > >>>> to
> > > > > >     > >>>>>> be identified as a `certificated member`. If not,
> this
> > > > > means there
> > > > > >     > >>> are
> > > > > >     > >>>>>> duplicate consumer members with same member name and
> > the
> > > > > request
> > > > > >     > >> will
> > > > > >     > >>>> be
> > > > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> > > client
> > > > > will
> > > > > >     > >> send a
> > > > > >     > >>>>>> leave group request (static membership mode). In
> > static
> > > > > membership,
> > > > > >     > >>> we
> > > > > >     > >>>>> will
> > > > > >     > >>>>>> also define `change-group-timeout` to hold on
> > rebalance
> > > > > provided by
> > > > > >     > >>>>> leader.
> > > > > >     > >>>>>> So we will wait for all the members to rejoin the
> > group
> > > > and
> > > > > do
> > > > > >     > >>> exactly
> > > > > >     > >>>>> one
> > > > > >     > >>>>>> rebalance since all members are expected to rejoin
> > > within
> > > > > timeout.
> > > > > >     > >> If
> > > > > >     > >>>>>> consumer crashes, the join group request from the
> > > > restarted
> > > > > >     > >> consumer
> > > > > >     > >>>> will
> > > > > >     > >>>>>> be recognized as an existing member and be handled
> as
> > > > above
> > > > > >     > >> condition
> > > > > >     > >>>> 1;
> > > > > >     > >>>>>> However, if the consumer takes longer than session
> > > timeout
> > > > > to
> > > > > >     > >> return,
> > > > > >     > >>>> we
> > > > > >     > >>>>>> shall still trigger rebalance but it could still try
> > to
> > > > > catch
> > > > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > > > timeout, its
> > > > > >     > >>>> cached
> > > > > >     > >>>>>> state on broker will be garbage collected and
> trigger
> > a
> > > > new
> > > > > >     > >> rebalance
> > > > > >     > >>>>> when
> > > > > >     > >>>>>> it finally joins.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> And consider the switch between dynamic to static
> > > > > membership.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall
> revise
> > > the
> > > > > >     > >> membership
> > > > > >     > >>>> to
> > > > > >     > >>>>>> static and wait for all the current members to
> > restart,
> > > > > since their
> > > > > >     > >>>>>> membership is still dynamic. Here our assumption is
> > that
> > > > the
> > > > > >     > >> restart
> > > > > >     > >>>>>> process shouldn't take a long time, as long restart
> is
> > > > > breaking the
> > > > > >     > >>>>>> `rebalance timeout` in whatever membership protocol
> we
> > > are
> > > > > using.
> > > > > >     > >>>> Before
> > > > > >     > >>>>>> restart, all dynamic member join requests will be
> > > > rejected.
> > > > > >     > >>>>>>  2.  Static to dynamic: this is more like a
> downgrade
> > > > which
> > > > > should
> > > > > >     > >>> be
> > > > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > > > session
> > > > > timeout
> > > > > >     > >>> to
> > > > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > > > current
> > > > > >     > >>> behavior)
> > > > > >     > >>>>>>  3.  Halfway switch: a corner case is like some
> > clients
> > > > keep
> > > > > >     > >> dynamic
> > > > > >     > >>>>>> membership while some keep static membership. This
> > will
> > > > > cause the
> > > > > >     > >>> group
> > > > > >     > >>>>>> rebalance forever without progress because
> > > dynamic/static
> > > > > states
> > > > > >     > >> are
> > > > > >     > >>>>>> bouncing each other. This could guarantee that we
> will
> > > not
> > > > > make the
> > > > > >     > >>>>>> consumer group work in a wrong state by having half
> > > static
> > > > > and half
> > > > > >     > >>>>> dynamic.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> To guarantee correctness, we will also push the
> member
> > > > > name/id pair
> > > > > >     > >>> to
> > > > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out)
> and
> > > > > upgrade the
> > > > > >     > >> API
> > > > > >     > >>>>>> version, these details will be further discussed
> back
> > in
> > > > > the KIP.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Are there any concern for this high level proposal?
> > Just
> > > > > want to
> > > > > >     > >>>>> reiterate
> > > > > >     > >>>>>> on the core idea of the KIP: "If the broker
> recognize
> > > this
> > > > > consumer
> > > > > >     > >>> as
> > > > > >     > >>>> an
> > > > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Thanks a lot for everyone's input! I feel this
> > proposal
> > > is
> > > > > much
> > > > > >     > >> more
> > > > > >     > >>>>>> robust than previous one!
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Best,
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Boyang
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> ________________________________
> > > > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > > > >     > >>>>>> To: dev@kafka.apache.org
> > > > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple
> > consumer
> > > > > rebalances
> > > > > >     > >>> by
> > > > > >     > >>>>>> specifying member id
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Hi,
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> > > about
> > > > > internals
> > > > > >     > >>>> again
> > > > > >     > >>>>>> :)
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> I like the idea or a user config `member.name` and
> to
> > > > keep
> > > > > `
> > > > > >     > >>> member.id`
> > > > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > > > client.id`
> > > > > might
> > > > > >     > >>> not
> > > > > >     > >>>>>> be a good idea.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> To clarify the algorithm, each time we generate a
> new
> > `
> > > > > member.id`,
> > > > > >     > >>> we
> > > > > >     > >>>>>> also need to update the "group membership"
> information
> > > > (ie,
> > > > > mapping
> > > > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `
> > member.id
> > > `
> > > > > replaces
> > > > > >     > >>> the
> > > > > >     > >>>>>> old entry in the cache.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> I also think, we need to preserve the `member.name
> ->
> > > > > member.id`
> > > > > >     > >>>> mapping
> > > > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should
> > mention
> > > > > this IMHO.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> For changing the default value of config
> > > > > `leave.group.on.close`. I
> > > > > >     > >>>> agree
> > > > > >     > >>>>>> with John, that we should not change the default
> > config,
> > > > > because it
> > > > > >     > >>>>>> would impact all consumer groups with dynamic
> > > assignment.
> > > > > However,
> > > > > >     > >> I
> > > > > >     > >>>>>> think we can document, that if static assignment is
> > used
> > > > > (ie,
> > > > > >     > >>>>>> `member.name` is configured) we never send a
> > > > > LeaveGroupRequest
> > > > > >     > >>>>>> regardless of the config. Note, that the config is
> > > > > internal, so not
> > > > > >     > >>>> sure
> > > > > >     > >>>>>> how to document this in detail. We should not expose
> > the
> > > > > internal
> > > > > >     > >>>> config
> > > > > >     > >>>>>> in the docs.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> About upgrading: why do we need have two rolling
> > bounces
> > > > > and encode
> > > > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> If we upgrade an existing consumer group from
> dynamic
> > to
> > > > > static, I
> > > > > >     > >>>> don't
> > > > > >     > >>>>>> see any reason why both should not work together and
> > > > single
> > > > > rolling
> > > > > >     > >>>>>> bounce would not be sufficient? If we bounce the
> first
> > > > > consumer and
> > > > > >     > >>>>>> switch from dynamic to static, it sends a `
> > member.name`
> > > > > and the
> > > > > >     > >>> broker
> > > > > >     > >>>>>> registers the [member.name, member.id] in the
> cache.
> > > Why
> > > > > would
> > > > > >     > >> this
> > > > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > > > assignment?
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> Also, Guozhang mentioned that for all other request,
> > we
> > > > > need to
> > > > > >     > >> check
> > > > > >     > >>>> if
> > > > > >     > >>>>>> the mapping [member.name, member.id] contains the
> > send
> > > `
> > > > > member.id`
> > > > > >     > >>> --
> > > > > >     > >>>> I
> > > > > >     > >>>>>> don't think this is necessary -- it seems to be
> > > sufficient
> > > > > to check
> > > > > >     > >>> the
> > > > > >     > >>>>>> `member.id` from the [member.id, Assignment]
> mapping
> > as
> > > > be
> > > > > do
> > > > > >     > >> today
> > > > > >     > >>> --
> > > > > >     > >>>>>> thus, checking `member.id` does not require any
> > change
> > > > > IMHO.
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>> -Matthias
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > > > >     > >>>>>>> @James
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>> What you described is true: the transition from
> > dynamic
> > > > to
> > > > > static
> > > > > >     > >>>>>>> memberships are not thought through yet. But I do
> not
> > > > > think it is
> > > > > >     > >>> an
> > > > > >     > >>>>>>> impossible problem: note that we indeed moved the
> > > offset
> > > > > commit
> > > > > >     > >>> from
> > > > > >     > >>>> ZK
> > > > > >     > >>>>>> to
> > > > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is
> > to
> > > > > first to
> > > > > >     > >>>>>>> double-commits on both zk and coordinator, and then
> > do
> > > a
> > > > > second
> > > > > >     > >>> round
> > > > > >     > >>>>> to
> > > > > >     > >>>>>>> turn the zk off.
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > > > >     > >>>> two-rolling-bounce
> > > > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag
> > to
> > > > > "static"
> > > > > >     > >>> while
> > > > > >     > >>>>>> keep
> > > > > >     > >>>>>>> the registry-id field empty still, in this case,
> the
> > > > > coordinator
> > > > > >     > >>>> still
> > > > > >     > >>>>>>> follows the logic of "dynamic", accepting the
> request
> > > > while
> > > > > >     > >>> allowing
> > > > > >     > >>>>> the
> > > > > >     > >>>>>>> protocol to be set to "static"; after the first
> > rolling
> > > > > bounce,
> > > > > >     > >> the
> > > > > >     > >>>>> group
> > > > > >     > >>>>>>> protocol is already "static", then a second rolling
> > > > bounce
> > > > > is
> > > > > >     > >>>> triggered
> > > > > >     > >>>>>> and
> > > > > >     > >>>>>>> this time we set the registry-id.
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>> Guozhang
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > > > >     > >> wushujames@gmail.com>
> > > > > >     > >>>>>> wrote:
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> > > this:
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > > > wangguoz@gmail.com
> > > > > >     > >>>
> > > > > >     > >>>>> wrote:
> > > > > >     > >>>>>>>>>
> > > > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with
> additional
> > > > > fields:
> > > > > >     > >>>>>>>>>
> > > > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> > > membership
> > > > > >     > >>> protocols.
> > > > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > > > pre-defined
> > > > > >     > >>> member
> > > > > >     > >>>>> id.
> > > > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> > > optional
> > > > > >     > >>>>>>>>> "group-change-timeout" value.
> > > > > >     > >>>>>>>>>
> > > > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the
> > two
> > > > > protocols
> > > > > >     > >>> for
> > > > > >     > >>>>> all
> > > > > >     > >>>>>>>>> group members: we accept the protocol on the
> first
> > > > joined
> > > > > >     > >> member
> > > > > >     > >>> of
> > > > > >     > >>>>> the
> > > > > >     > >>>>>>>>> group, and if later joining members indicate a
> > > > different
> > > > > >     > >>> membership
> > > > > >     > >>>>>>>>> protocol, we reject it. If the
> group-change-timeout
> > > > > value was
> > > > > >     > >>>>> different
> > > > > >     > >>>>>>>> to
> > > > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > > > application that
> > > > > >     > >>>> wants
> > > > > >     > >>>>>> to
> > > > > >     > >>>>>>>> switch to using static membership? Let's say there
> > are
> > > > 10
> > > > > >     > >>> instances
> > > > > >     > >>>> of
> > > > > >     > >>>>>> it.
> > > > > >     > >>>>>>>> As the instances go through a rolling restart,
> they
> > > will
> > > > > switch
> > > > > >     > >>> from
> > > > > >     > >>>>>>>> dynamic membership (the default?) to static
> > > membership.
> > > > > As each
> > > > > >     > >>> one
> > > > > >     > >>>>>> leaves
> > > > > >     > >>>>>>>> the group and restarts, they will be rejected from
> > the
> > > > > group
> > > > > >     > >>>> (because
> > > > > >     > >>>>>> the
> > > > > >     > >>>>>>>> group is currently using dynamic membership). The
> > > group
> > > > > will
> > > > > >     > >>> shrink
> > > > > >     > >>>>> down
> > > > > >     > >>>>>>>> until there is 1 node handling all the traffic.
> > After
> > > > > that one
> > > > > >     > >>>>> restarts,
> > > > > >     > >>>>>>>> the group will switch over to static membership.
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>> Is that right? That means that the transition plan
> > > from
> > > > > dynamic
> > > > > >     > >> to
> > > > > >     > >>>>>> static
> > > > > >     > >>>>>>>> membership isn't very smooth.
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> > > This
> > > > > reminds
> > > > > >     > >> me
> > > > > >     > >>>> of
> > > > > >     > >>>>>> the
> > > > > >     > >>>>>>>> transition plans that were discussed for moving
> from
> > > > > >     > >>> zookeeper-based
> > > > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers.
> That
> > > was
> > > > > also
> > > > > >     > >>> hard,
> > > > > >     > >>>>> and
> > > > > >     > >>>>>>>> ultimately we decided not to build that.
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>> -James
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>>
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>>
> > > > > >     > >>>>>
> > > > > >     > >>>>
> > > > > >     > >>>>
> > > > > >     > >>>>
> > > > > >     > >>>> --
> > > > > >     > >>>> -- Guozhang
> > > > > >     > >>>>
> > > > > >     > >>>
> > > > > >     > >>
> > > > > >     > >>
> > > > > >     > >>
> > > > > >     > >> --
> > > > > >     > >> -- Guozhang
> > > > > >     > >>
> > > > > >     >
> > > > > >
> > > > > >
> > > > >
> > > > >
> > > >
> > > > --
> > > > -Regards,
> > > > Mayuresh R. Gharat
> > > > (862) 250-7125
> > > >
> > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thank you for the clarification Jason! The proposals make sense here and let me continue the discussion.

> Then the ids would be determined using some convention. Most likely, we would just use sequential numbers 0, 1, 2,
> etc. We do the same thing for partition ids.


If we initialize a set of member names (I assume ids = names here) on broker through Admin API, the client needs to pick up this information simultaneously which I doubt if there is a generic way to achieve that? It would also make the scaling operations difficult if we need to define the member names every time we change the member set which is an extra operation burden. From my daily ops experience, dynamically generate member names on client side would be easier. Is there a good approach to address this issue?

> I was thinking that the registration is specified ahead of time and remains valid until changed. It would be more like a
> replica assignment. We don't move partitions just because a broker is down.
> The expectation is that it the broker will eventually return.


At Pinterest we maintain 24-7 SLA for 15 - 30 minutes reaction to all the critical streaming services abnormality. One of the burden was the night shift which requires the oncaller to quickly resolve the issue and get the streaming application back on track, however there is a chance of miss. My concern was that if we forfeit the timeout on static membership to trigger rebalance, missing some pages during midnight could be negatively impacting the system performance since we may realize that some partitions stop working for a couple of hours already until next morning. So registration timeout serves as the "last line of defense" to guarantee liveness if no human intervention jumps in.


I'm very interested in the replication protocol currently implemented on Kafka, but I'm not familiar with it. If we do have mechanism to handle issues like I mentioned above for replication (auto healing during mid-night if one broker is never back), we could continue discussing the new approaches to have basic guarantee of consumer group liveness.


The discussion so far is to make sure that all the design approaches we have taken are pointing to real scenarios. Once we clarify the scenarios, we would definitely propose better solution on top of it. I hope these discussions make sense. Thanks again for helping make the design solid!


Boyang

________________________________
From: Jason Gustafson <ja...@confluent.io>
Sent: Thursday, November 15, 2018 9:54 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

>
> I feel this would make the current protocol harder to use. For example, on
> KStream we typically would expect (number of instances * number of threads
> per instance) consumers. Giving out all this many member names in a list
> may not be easy, compared with dynamic generation of member names, at least
> for KStream use case.


That's a fair point. What I had in mind is for the API to specify the
number of consumers in the group. Then the ids would be determined using
some convention. Most likely, we would just use sequential numbers 0, 1, 2,
etc. We do the same thing for partition ids. Streams folks can chime in and
say whether that would work or not.

So this suggests we will not rely on heartbeat and commit interval to
> trigger rebalance? Even in static membership, I feel tracking active
> members is still required to be handled by broker since not all users are
> fully equipped with monitoring tools, otherwise users will feel
> uncomfortable using static membership solely for reducing rebalance purpose.


Let me clarify. I think it is still useful to distinguish the liveness of
the consumer using the heartbeat mechanism and to propagate that
information to the leader during rebalances. This gives the group leader
the option (but not the requirement) to change the partition assignment for
inactive members. My suggestion was more about how long the static
registration would remain valid. In the current proposal we have a
registration timeout, and I think Mayuresh is fair to point out the
potential confusion with the session timeout. It's kind of the same, but
not exactly. So instead, I was thinking that the registration is specified
ahead of time and remains valid until changed. It would be more like a
replica assignment. We don't move partitions just because a broker is down.
The expectation is that it the broker will eventually return. Similarly for
stateful applications, we would have the option to do the same thing. The
registration is fixed until someone changes it.

Does that make sense?

Thanks,
Jason


On Wed, Nov 14, 2018 at 2:46 PM, Boyang Chen <bc...@outlook.com> wrote:

> Thanks Jason for the suggestions! I update the KIP with full schema
> changes.
>
> > we offer an admin API that lets a user define the expected members of
> the group.
>
> I feel this would make the current protocol harder to use. For example, on
> KStream we typically would expect (number of instances * number of threads
> per instance) consumers. Giving out all this many member names in a list
> may not be easy, compared with dynamic generation of member names, at least
> for KStream use case.
>
> > For the sake of discussion, I was wondering if we could just say that
> static members do not expire.
>
>
> So this suggests we will not rely on heartbeat and commit interval to
> trigger rebalance? Even in static membership, I feel tracking active
> members is still required to be handled by broker since not all users are
> fully equipped with monitoring tools, otherwise users will feel
> uncomfortable using static membership solely for reducing rebalance purpose.
>
>
> For the admin API design, I'm simplifying the join group request handling,
> while using admin tool to switch between static and dynamic membership and
> set the two corresponding timeouts. Do you think this approach makes sense?
> The version one implementation will be much more clean if we handle
> membership change through user intervention.
>
>
> Best,
>
> Boyang
>
> ________________________________
> From: Jason Gustafson <ja...@confluent.io>
> Sent: Wednesday, November 14, 2018 9:31 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey Boyang,
>
> Thanks for the updates. From a high level, I think this actually
> complements Konstantine's writeup on incremental rebalancing. The gap we're
> addressing is providing a way to bind the the partition assignment of a
> group to a set of user-provided ids so that we are not so reliant on the
> group's immediate state. For example, these ids might identify the state
> store volume for particular streams instances. This is basically what you
> need to work well with k8s stateful sets (as far as I understand them).
>
> One key decision is how we would define and update the expected static
> members in a consumer group. The mechanics of the registration and
> expansion timeouts feel a little bit clunky. For the sake of discussion, I
> was wondering if we could just say that static members do not expire.
> Instead, we offer an admin API that lets a user define the expected members
> of the group. This API could be used to both grow and shrink a group. This
> would solve the rebalancing problems when applications are initially
> bootstrapped or when they are restarted because we would always know how
> many members should be in a group. What do you think?
>
> By the way, it would be helpful to include the full schema definition for
> any protocol changes in the proposal.
>
> Thanks,
> Jason
>
>
> On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Mayuresh for the feedback! Do you have a quick example for passing
> > in consumer config dynamically? I mainly use Kafka Streams at my daily
> work
> > so probably missing the idea how to do it in the current consumer
> setting.
> >
> >
> > For differentiating session timeout and registration timeout, I would try
> > to enhance the documentation in the first stage to see how people react
> to
> > the confusion (would be great if they feel straightforward!). Since one
> > doesn't have to fully understand the difference unless defining the new
> > config "member name", for current users we could buy some time to listen
> to
> > their understandings and improve our documentation correspondingly in the
> > follow-up KIPs.
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Mayuresh Gharat <gh...@gmail.com>
> > Sent: Sunday, November 11, 2018 1:06 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the reply.
> >
> > Please find the replies inline below :
> > For having a consumer config at runtime, I think it's not necessary to
> > address in this KIP because most companies run sidecar jobs through
> daemon
> > software like puppet. It should be easy to change the config through
> script
> > or UI without actual code change. We still want to leave flexibility for
> > user to define member name as they like.
> > ---- This might be little different for companies that use configuration
> > management tools that does not allow the applications to define/change
> the
> > configs dynamically. For example, if we use something similar to spring
> to
> > pull in the configs for the KafkaConsumer and pass it to the constructor
> to
> > create the KafkaConsumer object, it will be hard to specify a unique
> value
> > to the "MEMBER_NAME" config unless someone deploying the app generates a
> > unique string for this config outside the deployment workflow and copies
> it
> > statically before starting up each consumer instance. Unless we can
> loosen
> > the criteria for uniqueness of this config value, for each consumer
> > instance in the consumer group, I am not sure of a better way of
> > addressing this. If we don't want to loosen the criteria, then providing
> a
> > dynamic way to pass this in at runtime, would put the onus of having the
> > same unique value each time a consumer is restarted, on to the
> application
> > that is running the consumer.
> >
> > I just updated the kip about having both "registration timeout" and
> > "session timeout". The benefit of having two configs instead of one is to
> > reduce the mental burden for operation, for example user just needs to
> > unset "member name" to cast back to dynamic membership without worrying
> > about tuning the "session timeout" back to a smaller value.
> > --- That is a good point. I was thinking, if both the configs are
> > specified, it would be confusing for the end user without understanding
> the
> > internals of the consumer and its interaction with group coordinator, as
> > which takes precedence when and how it affects the consumer behavior.
> Just
> > my 2 cents.
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Hey Mayuresh,
> > >
> > >
> > > thanks for the thoughtful questions! Let me try to answer your
> questions
> > > one by one.
> > >
> > >
> > > For having a consumer config at runtime, I think it's not necessary to
> > > address in this KIP because most companies run sidecar jobs through
> > daemon
> > > software like puppet. It should be easy to change the config through
> > script
> > > or UI without actual code change. We still want to leave flexibility
> for
> > > user to define member name as they like.
> > >
> > >
> > > I just updated the kip about having both "registration timeout" and
> > > "session timeout". The benefit of having two configs instead of one is
> to
> > > reduce the mental burden for operation, for example user just needs to
> > > unset "member name" to cast back to dynamic membership without worrying
> > > about tuning the "session timeout" back to a smaller value.
> > >
> > >
> > > For backup topic, I think it's a low-level detail which could be
> > addressed
> > > in the implementation. I feel no preference of adding a new topic vs
> > reuse
> > > consumer offsets topic. I will do more analysis and make a trade-off
> > > comparison. Nice catch!
> > >
> > >
> > > I hope the explanations make sense to you. I will keep polishing on the
> > > edge cases and details.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Mayuresh Gharat <gh...@gmail.com>
> > > Sent: Saturday, November 10, 2018 10:25 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the KIP and sorry for being late to the party. This KIP is
> > > really useful for us at Linkedin.
> > >
> > > I had a few questions :
> > >
> > > The idea of having static member name seems nice, but instead of a
> > config,
> > > would it be possible for it to be passed in to the consumer at runtime?
> > > This is because an app might want to decide the config value at runtime
> > > using its host information for example, to generate the unique member
> > name.
> > >
> > > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > > wondering if we can reuse the session timeout here. This might help us
> to
> > > have one less config on the consumer.
> > >
> > > The KIP also talks about adding another internal topic
> > "static_member_map".
> > > Would the semantics (GroupCoordinator broker, topic configs) be the
> same
> > as
> > > __consumer_offsets topic?
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > >
> > > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > > > I took a quick pass of the proposal. First I would say it's a very
> > > > brilliant initiative from Konstantine and Confluent folks. To draft
> up
> > a
> > > > proposal like this needs deep understanding of the rebalance
> protocol!
> > I
> > > > summarized some thoughts here.
> > > >
> > > >
> > > > Overall the motivations of the two proposals align on that:
> > > >
> > > >   1.  Both believe the invariant resource (belonging to the same
> > process)
> > > > should be preserved across rebalance.
> > > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > > redistribution. I don't use rebalance here since part one of the
> > > > cooperative proposal could potentially introduce more rebalances but
> > only
> > > > on must-move resources.
> > > >   3.  Scale up/down and rolling bounce are causing unnecessary
> resource
> > > > shuffling that need to be mitigated.
> > > >
> > > >
> > > > On motivation level, I think both approach could solve/mitigate the
> > above
> > > > issues. They are just different in design philosophy, or I would say
> > the
> > > > perspective difference between framework user and algorithm designer.
> > > >
> > > >
> > > > Two proposals have different focuses. KIP-345 is trying to place more
> > > > fine-grained control on the broker side to reduce the unnecessary
> > > > rebalances, while keeping the client logic intact. This is pretty
> > > intuitive
> > > > cause-effect for normal developers who are not very familiar with
> > > rebalance
> > > > protocol. As a developer working with Kafka Streams daily, I'd be
> happy
> > > to
> > > > see a simplified rebalance protocol and just focus on maintaining the
> > > > stream/consumer jobs. Too many rebalances raised my concern on the
> job
> > > > health. To be concise, static membership has the advantage of
> reducing
> > > > mental burden.
> > > >
> > > >
> > > > Cooperative proposal takes thoughtful approach on client side. We
> want
> > to
> > > > have fine-grained control on the join/exit group behaviors and make
> the
> > > > current dynamic membership better to address above issues. I do feel
> > our
> > > > idea crossed on the delayed rebalance when we scale up/down, which
> > could
> > > > potentially reduce the state shuffling and decouple the behavior from
> > > > session timeout which is already overloaded.  In this sense, I
> believe
> > > both
> > > > approaches would serve well in making "reasonable rebalance" happen
> at
> > > the
> > > > "right timing".
> > > >
> > > >
> > > > However, based on my understanding, either 345 or cooperative
> > rebalancing
> > > > is not solving the problem Mike has proposed: could we do a better
> job
> > at
> > > > scaling up/down in ideal timing? My initial response was to introduce
> > an
> > > > admin API which now I feel is sub-optimal, in that the goal of smooth
> > > > transition is to make sure the newly up hosts are actually "ready".
> For
> > > > example:
> > > >
> > > >
> > > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At
> some
> > > > time we would like to scale up to 8 hosts, with the current
> > improvements
> > > we
> > > > could reduce 4 potential rebalances to a single one. But the new
> hosts
> > > are
> > > > yet unknown to be "ready" if they need to reconstruct the local
> state.
> > To
> > > > be actually ready, we need 4 standby tasks running on those empty
> hosts
> > > and
> > > > leader needs to wait for the signal of "replay/reconstruct complete"
> to
> > > > actually involve them into the main consumer group. Otherwise,
> > rebalance
> > > > just kills our performance since we need to wait indefinite long for
> > task
> > > > migration.
> > > >
> > > >
> > > > The scale down is also tricky such that we are not able to define a
> > > "true"
> > > > leave of a member. Rebalance immediately after "true" leaves are most
> > > > optimal comparing with human intervention. Does this make sense?
> > > >
> > > >
> > > > My intuition is that cooperative approach which was implemented on
> the
> > > > client side could better handle scaling cases than KIP 345, since it
> > > > involves a lot of algorithmic changes to define "replaying" stage,
> > which
> > > I
> > > > feel would over-complicate broker logic if implemented on
> coordinator.
> > If
> > > > we let 345 focus on reducing unnecessary rebalance, and let
> cooperative
> > > > approach focus on judging best timing of scale up/down, the two
> efforts
> > > > could be aligned. In long term, I feel the more complex improvement
> of
> > > > consumer protocol should happen on client side instead of server side
> > > which
> > > > is easier to test and has less global impact for the entire Kafka
> > > > production cluster.
> > > >
> > > >
> > > > Thanks again to Konstantine, Matthias and other folks in coming up
> with
> > > > this great client proposal. This is great complementation to KIP 345.
> > In
> > > a
> > > > high level, we are not having any collision on the path and both
> > > proposals
> > > > are making sense here. Just need better sync to avoid duplicate
> effort
> > :)
> > > >
> > > >
> > > > Best,
> > > >
> > > > Boyang
> > > >
> > > >
> > > > ________________________________
> > > > From: Boyang Chen <bc...@outlook.com>
> > > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > > > deeper look and make a comparison between the two proposals.
> > > >
> > > >
> > > > Meanwhile for the scale down specifically for stateful streaming, we
> > > could
> > > > actually introduce a new status called "learner" where the newly up
> > hosts
> > > > could try to catch up with the assigned task progress first before
> > > > triggering the rebalance, from which we don't see a sudden dip on the
> > > > progress. However, it is built on top of the success of KIP-345.
> > > >
> > > >
> > > > ________________________________
> > > > From: Matthias J. Sax <ma...@confluent.io>
> > > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Hey,
> > > >
> > > > there was quite a pause on this KIP discussion and in the mean time,
> a
> > > > new design for incremental cooporative rebalance was suggested:
> > > >
> > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > Incremental Cooperative Rebalancing: Support and Policies ...<
> > https://cwiki.apache.org/confluence/display/KAFKA/Increm
> > ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> > cwiki.apache.org
> > Rebalancing between distributed application processes in Apache Kafka was
> > enhanced considerably when it was decoupled as logic from Kafka brokers
> and
> > was moved as responsibility to the clients and specifically to Kafka
> > Consumer. This pattern has been working robustly for quite a while now
> and
> > has ...
> >
> >
> >
> > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > >
> > > cwiki.apache.org
> > > Rebalancing between distributed application processes in Apache Kafka
> was
> > > enhanced considerably when it was decoupled as logic from Kafka brokers
> > and
> > > was moved as responsibility to the clients and specifically to Kafka
> > > Consumer. This pattern has been working robustly for quite a while now
> > and
> > > has ...
> > >
> > >
> > >
> > > >
> > > >
> > > > We should make sure that the proposal and this KIP align to each
> other.
> > > > Thoughts?
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > > Hey Mike,
> > > > >
> > > > >
> > > > > thanks for the feedback, the two question are very thoughtful!
> > > > >
> > > > >
> > > > >> 1) I am a little confused about the distinction for the leader. If
> > the
> > > > consumer node that was assigned leader does a bounce (goes down and
> > > quickly
> > > > comes up) to update application code, will a rebalance be triggered?
> I
> > >
> > > do
> > > > not think a bounce of the leader should trigger a rebalance.
> > > > >
> > > > > For Q1 my intention was to minimize the change within one KIP,
> since
> > > the
> > > > leader rejoining case could be addressed separately.
> > > > >
> > > > >
> > > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > > gracefully increase the number of nodes in the cluster. I think we
> need
> > > to
> > > > support graceful shrink down as well. If I set the registration
> timeout
> > > to
> > > > 5 minutes > to handle rolling restarts or intermittent failures
> without
> > > > shuffling state, I don't want to wait 5 minutes in order for the
> group
> > to
> > > > rebalance if I am intentionally removing a node from the cluster. I
> am
> > > not
> > > > sure the best way to > do this. One idea I had was adding the ability
> > > for a
> > > > CLI or Admin API to force a rebalance of the group. This would allow
> > for
> > > an
> > > > admin to trigger the rebalance manually without waiting the entire
> > > > registration timeout on > shrink down. What do you think?
> > > > >
> > > > > For 2) my understanding is that for scaling down case it is better
> to
> > > be
> > > > addressed by CLI tool than code logic, since only by human evaluation
> > we
> > > > could decide whether it is a "right timing" -- the time when all the
> > > > scaling down consumers are offline -- to kick in rebalance. Unless we
> > > > introduce another term on coordinator which indicates the target
> > consumer
> > > > group size, broker will find it hard to decide when to start
> rebalance.
> > > So
> > > > far I prefer to hold the implementation for that, but agree we could
> > > > discuss whether we want to introduce admin API in this KIP or a
> > separate
> > > > one.
> > > > >
> > > > >
> > > > > Thanks again for the proposed ideas!
> > > > >
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Mike Freyberger <mi...@xandr.com>
> > > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > specifying member id
> > > > >
> > > > > Boyang,
> > > > >
> > > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > > >
> > > > > 1) I am a little confused about the distinction for the leader. If
> > the
> > > > consumer node that was assigned leader does a bounce (goes down and
> > > quickly
> > > > comes up) to update application code, will a rebalance be triggered?
> I
> > do
> > > > not think a bounce of the leader should trigger a rebalance.
> > > > >
> > > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > > gracefully increase the number of nodes in the cluster. I think we
> need
> > > to
> > > > support graceful shrink down as well. If I set the registration
> timeout
> > > to
> > > > 5 minutes to handle rolling restarts or intermittent failures without
> > > > shuffling state, I don't want to wait 5 minutes in order for the
> group
> > to
> > > > rebalance if I am intentionally removing a node from the cluster. I
> am
> > > not
> > > > sure the best way to do this. One idea I had was adding the ability
> > for a
> > > > CLI or Admin API to force a rebalance of the group. This would allow
> > for
> > > an
> > > > admin to trigger the rebalance manually without waiting the entire
> > > > registration timeout on shrink down. What do you think?
> > > > >
> > > > > Mike
> > > > >
> > > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > > > >
> > > > >     Btw, I updated KIP 345 based on my understanding. Feel free to
> > take
> > > > another round of look:
> > > > >
> > > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > KIP-345: Introduce static membership protocol to reduce ...<
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > >
> > > > > cwiki.apache.org
> > > > > For stateful applications, one of the biggest performance
> bottleneck
> > is
> > > > the state shuffling. In Kafka consumer, there is a concept called
> > > > "rebalance" which means that for given M partitions and N consumers
> in
> > > one
> > > > consumer group, Kafka will try to balance the load between consumers
> > and
> > > > ideally have ...
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > >
> > > > >     cwiki.apache.org
> > > > >     For stateful applications, one of the biggest performance
> > > bottleneck
> > > > is the state shuffling. In Kafka consumer, there is a concept called
> > > > "rebalance" which means that for given M partitions and N consumers
> in
> > > one
> > > > consumer group, Kafka will try to balance the load between consumers
> > and
> > > > ideally have ...
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >     ________________________________
> > > > >     From: Boyang Chen <bc...@outlook.com>
> > > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > > >     To: dev@kafka.apache.org
> > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by specifying member id
> > > > >
> > > > >     Thanks everyone for the input on this thread! (Sorry it's been
> a
> > > > while) I feel that we are very close to the final solution.
> > > > >
> > > > >
> > > > >     Hey Jason and Mike, I have two quick questions on the new
> > features
> > > > here:
> > > > >
> > > > >       1.  so our proposal is that until we add a new static member
> > into
> > > > the group (scale up), we will not trigger rebalance until the
> > > "registration
> > > > timeout"( the member has been offline for too long)? How about
> leader's
> > > > rejoin request, I think we should still trigger rebalance when that
> > > > happens, since the consumer group may have new topics to consume?
> > > > >       2.  I'm not very clear on the scale up scenario in static
> > > > membership here. Should we fallback to dynamic membership while
> > > > adding/removing hosts (by setting member.name = null), or we still
> > want
> > > > to add instances with `member.name` so that we eventually
> > expand/shrink
> > > > the static membership? I personally feel the easier solution is to
> spin
> > > up
> > > > new members and wait until either the same "registration timeout" or
> a
> > > > "scale up timeout" before starting the rebalance. What do you think?
> > > > >
> > > > >     Meanwhile I will go ahead to make changes to the KIP with our
> > newly
> > > > discussed items and details. Really excited to see the design has
> > become
> > > > more solid.
> > > > >
> > > > >     Best,
> > > > >     Boyang
> > > > >
> > > > >     ________________________________
> > > > >     From: Jason Gustafson <ja...@confluent.io>
> > > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > > >     To: dev
> > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by specifying member id
> > > > >
> > > > >     Hey Mike,
> > > > >
> > > > >     Yeah, that's a good point. A long "registration timeout" may
> not
> > be
> > > > a great
> > > > >     idea. Perhaps in practice you'd set it long enough to be able
> to
> > > > detect a
> > > > >     failure and provision a new instance. Maybe on the order of 10
> > > > minutes is
> > > > >     more reasonable.
> > > > >
> > > > >     In any case, it's probably a good idea to have an
> administrative
> > > way
> > > > to
> > > > >     force deregistration. One option is to extend the DeleteGroups
> > API
> > > > with a
> > > > >     list of members names.
> > > > >
> > > > >     -Jason
> > > > >
> > > > >
> > > > >
> > > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > > mfreyberger@appnexus.com>
> > > > >     wrote:
> > > > >
> > > > >     > Jason,
> > > > >     >
> > > > >     > Regarding step 4 in your proposal which suggests beginning a
> > long
> > > > timer
> > > > >     > (30 minutes) when a static member leaves the group, would
> there
> > > > also be the
> > > > >     > ability for an admin to force a static membership expiration?
> > > > >     >
> > > > >     > I'm thinking that during particular types of outages or
> > upgrades
> > > > users
> > > > >     > would want forcefully remove a static member from the group.
> > > > >     >
> > > > >     > So the user would shut the consumer down normally, which
> > wouldn't
> > > > trigger
> > > > >     > a rebalance. Then the user could use an admin CLI tool to
> force
> > > > remove that
> > > > >     > consumer from the group, so the TopicPartitions that were
> > > > previously owned
> > > > >     > by that consumer can be released.
> > > > >     >
> > > > >     > At a high level, we need consumer groups to gracefully handle
> > > > intermittent
> > > > >     > failures and permanent failures. Currently, the consumer
> group
> > > > protocol
> > > > >     > handles permanent failures well, but does not handle
> > intermittent
> > > > failures
> > > > >     > well (it creates unnecessary rebalances). I want to make sure
> > the
> > > > overall
> > > > >     > solution here handles both intermittent failures and
> permanent
> > > > failures,
> > > > >     > rather than sacrificing support for permanent failures in
> order
> > > to
> > > > provide
> > > > >     > support for intermittent failures.
> > > > >     >
> > > > >     > Mike
> > > > >     >
> > > > >     > Sent from my iPhone
> > > > >     >
> > > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > > jason@confluent.io>
> > > > wrote:
> > > > >     > >
> > > > >     > > Hey Guozhang,
> > > > >     > >
> > > > >     > > Responses below:
> > > > >     > >
> > > > >     > > Originally I was trying to kill more birds with one stone
> > with
> > > > KIP-345,
> > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > shutting
> > > > down a
> > > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > > email), and
> > > > >     > >> hence proposing to have a pure static-membership protocol.
> > But
> > > > thinking
> > > > >     > >> twice about it I now feel it may be too ambitious and
> worth
> > > > fixing in
> > > > >     > >> another KIP.
> > > > >     > >
> > > > >     > >
> > > > >     > > I was considering an extension to support
> pre-initialization
> > of
> > > > the
> > > > >     > static
> > > > >     > > members of the group, but I agree we should probably leave
> > this
> > > > problem
> > > > >     > for
> > > > >     > > future work.
> > > > >     > >
> > > > >     > > 1. How this longish static member expiration timeout
> defined?
> > > Is
> > > > it via a
> > > > >     > >> broker, hence global config, or via a client config which
> > can
> > > be
> > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > >     > >
> > > > >     > >
> > > > >     > > I am not too sure. I tend to lean toward server-side
> configs
> > > > because they
> > > > >     > > are easier to evolve. If we have to add something to the
> > > > protocol, then
> > > > >     > > we'll be stuck with it forever.
> > > > >     > >
> > > > >     > > 2. Assuming that for static members, LEAVE_GROUP request
> will
> > > not
> > > > >     > trigger a
> > > > >     > >> rebalance immediately either, similar to session timeout,
> > but
> > > > only the
> > > > >     > >> longer member expiration timeout, can we remove the
> > internal "
> > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > walk-around
> > > > >     > then?
> > > > >     > >
> > > > >     > >
> > > > >     > > Yeah, I hope we can ultimately get rid of it, but we may
> need
> > > it
> > > > for
> > > > >     > > compatibility with older brokers. A related question is
> what
> > > > should be
> > > > >     > the
> > > > >     > > behavior of the consumer if `member.name` is provided but
> > the
> > > > broker
> > > > >     > does
> > > > >     > > not support it? We could either fail or silently downgrade
> to
> > > > dynamic
> > > > >     > > membership.
> > > > >     > >
> > > > >     > > -Jason
> > > > >     > >
> > > > >     > >
> > > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > >     > wrote:
> > > > >     > >>
> > > > >     > >> Hey Jason,
> > > > >     > >>
> > > > >     > >> I like your idea to simplify the upgrade protocol to allow
> > > > co-exist of
> > > > >     > >> static and dynamic members. Admittedly it may make the
> > > > coordinator-side
> > > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > > >     > >>
> > > > >     > >> Originally I was trying to kill more birds with one stone
> > with
> > > > KIP-345,
> > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > shutting
> > > > down a
> > > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > > email), and
> > > > >     > >> hence proposing to have a pure static-membership protocol.
> > But
> > > > thinking
> > > > >     > >> twice about it I now feel it may be too ambitious and
> worth
> > > > fixing in
> > > > >     > >> another KIP. With that, I think what you've proposed here
> > is a
> > > > good way
> > > > >     > to
> > > > >     > >> go for KIP-345 itself.
> > > > >     > >>
> > > > >     > >> Note there are a few details in your proposal we'd still
> > need
> > > > to figure
> > > > >     > >> out:
> > > > >     > >>
> > > > >     > >> 1. How this longish static member expiration timeout
> > defined?
> > > > Is it via
> > > > >     > a
> > > > >     > >> broker, hence global config, or via a client config which
> > can
> > > be
> > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > >     > >>
> > > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> > will
> > > > not
> > > > >     > trigger a
> > > > >     > >> rebalance immediately either, similar to session timeout,
> > but
> > > > only the
> > > > >     > >> longer member expiration timeout, can we remove the
> > internal "
> > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > walk-around
> > > > >     > then?
> > > > >     > >>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> Guozhang
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > > jason@confluent.io>
> > > > >     > >> wrote:
> > > > >     > >>
> > > > >     > >>> Hey All,
> > > > >     > >>>
> > > > >     > >>> Nice to see some solid progress on this. It sounds like
> one
> > > of
> > > > the
> > > > >     > >>> complications is allowing static and dynamic registration
> > to
> > > > coexist.
> > > > >     > I'm
> > > > >     > >>> wondering if we can do something like the following:
> > > > >     > >>>
> > > > >     > >>> 1. Statically registered members (those joining the group
> > > with
> > > > a
> > > > >     > >> non-null `
> > > > >     > >>> member.name`) maintain a session with the coordinator
> just
> > > > like
> > > > >     > dynamic
> > > > >     > >>> members.
> > > > >     > >>> 2. If a session is active for a static member when a
> > > rebalance
> > > > begins,
> > > > >     > >> then
> > > > >     > >>> basically we'll keep the current behavior. The rebalance
> > will
> > > > await the
> > > > >     > >>> static member joining the group.
> > > > >     > >>> 3. If a static member does not have an active session,
> then
> > > the
> > > > >     > >> coordinator
> > > > >     > >>> will not wait for it to join, but will still include it
> in
> > > the
> > > > >     > rebalance.
> > > > >     > >>> The coordinator will forward the cached subscription
> > > > information to the
> > > > >     > >>> leader and will cache the assignment after the rebalance
> > > > completes.
> > > > >     > (Note
> > > > >     > >>> that we still have the generationId to fence offset
> commits
> > > > from a
> > > > >     > static
> > > > >     > >>> zombie if the assignment changes.)
> > > > >     > >>> 4. When a static member leaves the group or has its
> session
> > > > expire, no
> > > > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > > > expire the
> > > > >     > >> static
> > > > >     > >>> registration. This would be a longish timeout (like 30
> > > minutes
> > > > say).
> > > > >     > >>>
> > > > >     > >>> So basically static members participate in all rebalances
> > > > regardless
> > > > >     > >>> whether they have an active session. In a given
> rebalance,
> > > > some of the
> > > > >     > >>> members may be static and some dynamic. The group leader
> > can
> > > > >     > >> differentiate
> > > > >     > >>> the two based on the presence of the `member.name` (we
> > have
> > > > to add
> > > > >     > this
> > > > >     > >> to
> > > > >     > >>> the JoinGroupResponse). Generally speaking, we would
> choose
> > > > leaders
> > > > >     > >>> preferentially from the active members that support the
> > > latest
> > > > >     > JoinGroup
> > > > >     > >>> protocol and are using static membership. If we have to
> > > choose
> > > > a leader
> > > > >     > >>> with an old version, however, it would see all members in
> > the
> > > > group
> > > > >     > >> (static
> > > > >     > >>> or dynamic) as dynamic members and perform the assignment
> > as
> > > > usual.
> > > > >     > >>>
> > > > >     > >>> Would that work?
> > > > >     > >>>
> > > > >     > >>> -Jason
> > > > >     > >>>
> > > > >     > >>>
> > > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > >     > >> wrote:
> > > > >     > >>>
> > > > >     > >>>> Hello Boyang,
> > > > >     > >>>>
> > > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > > >     > >>>>
> > > > >     > >>>> 1. Where will "change-group-timeout" be communicated to
> > the
> > > > broker?
> > > > >     > >> Will
> > > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> > going
> > > > to
> > > > >     > >>> piggy-back
> > > > >     > >>>> on the existing session-timeout field (assuming that the
> > > > original
> > > > >     > value
> > > > >     > >>>> will not be used anywhere in the static membership any
> > > more)?
> > > > >     > >>>>
> > > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > > timeout to
> > > > >     > >> return,
> > > > >     > >>>> we shall still trigger rebalance but it could still try
> to
> > > > catch
> > > > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> > > your
> > > > proposal
> > > > >     > >> is
> > > > >     > >>>> that for static memberships, the broker will NOT trigger
> > > > rebalance
> > > > >     > even
> > > > >     > >>>> after session-timeout has been detected, but only that
> > after
> > > > >     > >>>> change-group-timeout
> > > > >     > >>>> which is supposed to be longer than session-timeout to
> be
> > > > defined?
> > > > >     > >>>>
> > > > >     > >>>> 3. "A join group request with member.name set will be
> > > > treated as
> > > > >     > >>>> `static-membership` strategy", in this case, how would
> the
> > > > switch from
> > > > >     > >>>> dynamic to static happen, since whoever changed the
> > > > member.name to
> > > > >     > >>>> not-null
> > > > >     > >>>> will be rejected, right?
> > > > >     > >>>>
> > > > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > > > timeout to
> > > > >     > >>> trigger
> > > > >     > >>>> rebalance should be sufficient." this is also a bit
> > unclear
> > > > to me: who
> > > > >     > >>> will
> > > > >     > >>>> erase the cached mapping? Since it is on the
> broker-side I
> > > > assume that
> > > > >     > >>>> broker has to do it. Are you suggesting to use a new
> > request
> > > > for it?
> > > > >     > >>>>
> > > > >     > >>>> 5. "Halfway switch": following 3) above, if your
> proposal
> > is
> > > > basically
> > > > >     > >> to
> > > > >     > >>>> let "first join-request wins", and the strategy will
> stay
> > as
> > > > is until
> > > > >     > >> all
> > > > >     > >>>> members are gone, then this will also not happen since
> > > > whoever used
> > > > >     > >>>> different strategy as the first guy who sends join-group
> > > > request will
> > > > >     > >> be
> > > > >     > >>>> rejected right?
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> Guozhang
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > > john@confluent.io>
> > > > >     > >> wrote:
> > > > >     > >>>>
> > > > >     > >>>>> This sounds good to me!
> > > > >     > >>>>>
> > > > >     > >>>>> Thanks for the time you've spent on it,
> > > > >     > >>>>> -John
> > > > >     > >>>>>
> > > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > > bchen11@outlook.com>
> > > > >     > >>>> wrote:
> > > > >     > >>>>>
> > > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy
> recently
> > > and
> > > > >     > >> haven't
> > > > >     > >>>> got
> > > > >     > >>>>>> time to update this thread. To summarize what we come
> up
> > > so
> > > > far,
> > > > >     > >> here
> > > > >     > >>>> is
> > > > >     > >>>>> a
> > > > >     > >>>>>> draft updated plan:
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Introduce a new config called `member.name` which is
> > > > supposed to
> > > > >     > >> be
> > > > >     > >>>>>> provided uniquely by the consumer client. The broker
> > will
> > > > maintain
> > > > >     > >> a
> > > > >     > >>>>> cache
> > > > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > > > request with
> > > > >     > >>>>>> member.name set will be treated as
> `static-membership`
> > > > strategy,
> > > > >     > >> and
> > > > >     > >>>>> will
> > > > >     > >>>>>> reject any join group request without member.name. So
> > > this
> > > > >     > >>>> coordination
> > > > >     > >>>>>> change will be differentiated from the
> > > `dynamic-membership`
> > > > >     > >> protocol
> > > > >     > >>> we
> > > > >     > >>>>>> currently have.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When handling static join group request:
> > > > >     > >>>>>>
> > > > >     > >>>>>>  1.   The broker will check the membership to see
> > whether
> > > > this is
> > > > >     > >> a
> > > > >     > >>>> new
> > > > >     > >>>>>> member. If new, broker allocate a unique member id,
> > cache
> > > > the
> > > > >     > >> mapping
> > > > >     > >>>> and
> > > > >     > >>>>>> move to rebalance stage.
> > > > >     > >>>>>>  2.   Following 1, if this is an existing member,
> broker
> > > > will not
> > > > >     > >>>> change
> > > > >     > >>>>>> group state, and return its cached member.id and
> > current
> > > > >     > >> assignment.
> > > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin
> > with
> > > > pair
> > > > >     > >>> member
> > > > >     > >>>>>> name and id, I think for join group request it is ok
> to
> > > > leave
> > > > >     > >> member
> > > > >     > >>> id
> > > > >     > >>>>>> blank as member name is the unique identifier. In
> commit
> > > > offset
> > > > >     > >>> request
> > > > >     > >>>>> we
> > > > >     > >>>>>> *must* have both.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When handling commit offset request, if enabled with
> > > static
> > > > >     > >>> membership,
> > > > >     > >>>>>> each time the commit request must have both
> member.name
> > > and
> > > > >     > >>> member.id
> > > > >     > >>>> to
> > > > >     > >>>>>> be identified as a `certificated member`. If not, this
> > > > means there
> > > > >     > >>> are
> > > > >     > >>>>>> duplicate consumer members with same member name and
> the
> > > > request
> > > > >     > >> will
> > > > >     > >>>> be
> > > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> > client
> > > > will
> > > > >     > >> send a
> > > > >     > >>>>>> leave group request (static membership mode). In
> static
> > > > membership,
> > > > >     > >>> we
> > > > >     > >>>>> will
> > > > >     > >>>>>> also define `change-group-timeout` to hold on
> rebalance
> > > > provided by
> > > > >     > >>>>> leader.
> > > > >     > >>>>>> So we will wait for all the members to rejoin the
> group
> > > and
> > > > do
> > > > >     > >>> exactly
> > > > >     > >>>>> one
> > > > >     > >>>>>> rebalance since all members are expected to rejoin
> > within
> > > > timeout.
> > > > >     > >> If
> > > > >     > >>>>>> consumer crashes, the join group request from the
> > > restarted
> > > > >     > >> consumer
> > > > >     > >>>> will
> > > > >     > >>>>>> be recognized as an existing member and be handled as
> > > above
> > > > >     > >> condition
> > > > >     > >>>> 1;
> > > > >     > >>>>>> However, if the consumer takes longer than session
> > timeout
> > > > to
> > > > >     > >> return,
> > > > >     > >>>> we
> > > > >     > >>>>>> shall still trigger rebalance but it could still try
> to
> > > > catch
> > > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > > timeout, its
> > > > >     > >>>> cached
> > > > >     > >>>>>> state on broker will be garbage collected and trigger
> a
> > > new
> > > > >     > >> rebalance
> > > > >     > >>>>> when
> > > > >     > >>>>>> it finally joins.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> And consider the switch between dynamic to static
> > > > membership.
> > > > >     > >>>>>>
> > > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise
> > the
> > > > >     > >> membership
> > > > >     > >>>> to
> > > > >     > >>>>>> static and wait for all the current members to
> restart,
> > > > since their
> > > > >     > >>>>>> membership is still dynamic. Here our assumption is
> that
> > > the
> > > > >     > >> restart
> > > > >     > >>>>>> process shouldn't take a long time, as long restart is
> > > > breaking the
> > > > >     > >>>>>> `rebalance timeout` in whatever membership protocol we
> > are
> > > > using.
> > > > >     > >>>> Before
> > > > >     > >>>>>> restart, all dynamic member join requests will be
> > > rejected.
> > > > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> > > which
> > > > should
> > > > >     > >>> be
> > > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > > session
> > > > timeout
> > > > >     > >>> to
> > > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > > current
> > > > >     > >>> behavior)
> > > > >     > >>>>>>  3.  Halfway switch: a corner case is like some
> clients
> > > keep
> > > > >     > >> dynamic
> > > > >     > >>>>>> membership while some keep static membership. This
> will
> > > > cause the
> > > > >     > >>> group
> > > > >     > >>>>>> rebalance forever without progress because
> > dynamic/static
> > > > states
> > > > >     > >> are
> > > > >     > >>>>>> bouncing each other. This could guarantee that we will
> > not
> > > > make the
> > > > >     > >>>>>> consumer group work in a wrong state by having half
> > static
> > > > and half
> > > > >     > >>>>> dynamic.
> > > > >     > >>>>>>
> > > > >     > >>>>>> To guarantee correctness, we will also push the member
> > > > name/id pair
> > > > >     > >>> to
> > > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > > > upgrade the
> > > > >     > >> API
> > > > >     > >>>>>> version, these details will be further discussed back
> in
> > > > the KIP.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Are there any concern for this high level proposal?
> Just
> > > > want to
> > > > >     > >>>>> reiterate
> > > > >     > >>>>>> on the core idea of the KIP: "If the broker recognize
> > this
> > > > consumer
> > > > >     > >>> as
> > > > >     > >>>> an
> > > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > > >     > >>>>>>
> > > > >     > >>>>>> Thanks a lot for everyone's input! I feel this
> proposal
> > is
> > > > much
> > > > >     > >> more
> > > > >     > >>>>>> robust than previous one!
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Best,
> > > > >     > >>>>>>
> > > > >     > >>>>>> Boyang
> > > > >     > >>>>>>
> > > > >     > >>>>>> ________________________________
> > > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > > >     > >>>>>> To: dev@kafka.apache.org
> > > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple
> consumer
> > > > rebalances
> > > > >     > >>> by
> > > > >     > >>>>>> specifying member id
> > > > >     > >>>>>>
> > > > >     > >>>>>> Hi,
> > > > >     > >>>>>>
> > > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> > about
> > > > internals
> > > > >     > >>>> again
> > > > >     > >>>>>> :)
> > > > >     > >>>>>>
> > > > >     > >>>>>> I like the idea or a user config `member.name` and to
> > > keep
> > > > `
> > > > >     > >>> member.id`
> > > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > > client.id`
> > > > might
> > > > >     > >>> not
> > > > >     > >>>>>> be a good idea.
> > > > >     > >>>>>>
> > > > >     > >>>>>> To clarify the algorithm, each time we generate a new
> `
> > > > member.id`,
> > > > >     > >>> we
> > > > >     > >>>>>> also need to update the "group membership" information
> > > (ie,
> > > > mapping
> > > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `
> member.id
> > `
> > > > replaces
> > > > >     > >>> the
> > > > >     > >>>>>> old entry in the cache.
> > > > >     > >>>>>>
> > > > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > > > member.id`
> > > > >     > >>>> mapping
> > > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should
> mention
> > > > this IMHO.
> > > > >     > >>>>>>
> > > > >     > >>>>>> For changing the default value of config
> > > > `leave.group.on.close`. I
> > > > >     > >>>> agree
> > > > >     > >>>>>> with John, that we should not change the default
> config,
> > > > because it
> > > > >     > >>>>>> would impact all consumer groups with dynamic
> > assignment.
> > > > However,
> > > > >     > >> I
> > > > >     > >>>>>> think we can document, that if static assignment is
> used
> > > > (ie,
> > > > >     > >>>>>> `member.name` is configured) we never send a
> > > > LeaveGroupRequest
> > > > >     > >>>>>> regardless of the config. Note, that the config is
> > > > internal, so not
> > > > >     > >>>> sure
> > > > >     > >>>>>> how to document this in detail. We should not expose
> the
> > > > internal
> > > > >     > >>>> config
> > > > >     > >>>>>> in the docs.
> > > > >     > >>>>>>
> > > > >     > >>>>>> About upgrading: why do we need have two rolling
> bounces
> > > > and encode
> > > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > > >     > >>>>>>
> > > > >     > >>>>>> If we upgrade an existing consumer group from dynamic
> to
> > > > static, I
> > > > >     > >>>> don't
> > > > >     > >>>>>> see any reason why both should not work together and
> > > single
> > > > rolling
> > > > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > > > consumer and
> > > > >     > >>>>>> switch from dynamic to static, it sends a `
> member.name`
> > > > and the
> > > > >     > >>> broker
> > > > >     > >>>>>> registers the [member.name, member.id] in the cache.
> > Why
> > > > would
> > > > >     > >> this
> > > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > > assignment?
> > > > >     > >>>>>>
> > > > >     > >>>>>> Also, Guozhang mentioned that for all other request,
> we
> > > > need to
> > > > >     > >> check
> > > > >     > >>>> if
> > > > >     > >>>>>> the mapping [member.name, member.id] contains the
> send
> > `
> > > > member.id`
> > > > >     > >>> --
> > > > >     > >>>> I
> > > > >     > >>>>>> don't think this is necessary -- it seems to be
> > sufficient
> > > > to check
> > > > >     > >>> the
> > > > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping
> as
> > > be
> > > > do
> > > > >     > >> today
> > > > >     > >>> --
> > > > >     > >>>>>> thus, checking `member.id` does not require any
> change
> > > > IMHO.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> -Matthias
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > > >     > >>>>>>> @James
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> What you described is true: the transition from
> dynamic
> > > to
> > > > static
> > > > >     > >>>>>>> memberships are not thought through yet. But I do not
> > > > think it is
> > > > >     > >>> an
> > > > >     > >>>>>>> impossible problem: note that we indeed moved the
> > offset
> > > > commit
> > > > >     > >>> from
> > > > >     > >>>> ZK
> > > > >     > >>>>>> to
> > > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is
> to
> > > > first to
> > > > >     > >>>>>>> double-commits on both zk and coordinator, and then
> do
> > a
> > > > second
> > > > >     > >>> round
> > > > >     > >>>>> to
> > > > >     > >>>>>>> turn the zk off.
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > > >     > >>>> two-rolling-bounce
> > > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag
> to
> > > > "static"
> > > > >     > >>> while
> > > > >     > >>>>>> keep
> > > > >     > >>>>>>> the registry-id field empty still, in this case, the
> > > > coordinator
> > > > >     > >>>> still
> > > > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> > > while
> > > > >     > >>> allowing
> > > > >     > >>>>> the
> > > > >     > >>>>>>> protocol to be set to "static"; after the first
> rolling
> > > > bounce,
> > > > >     > >> the
> > > > >     > >>>>> group
> > > > >     > >>>>>>> protocol is already "static", then a second rolling
> > > bounce
> > > > is
> > > > >     > >>>> triggered
> > > > >     > >>>>>> and
> > > > >     > >>>>>>> this time we set the registry-id.
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> Guozhang
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > > >     > >> wushujames@gmail.com>
> > > > >     > >>>>>> wrote:
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> > this:
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > >     > >>>
> > > > >     > >>>>> wrote:
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > > > fields:
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> > membership
> > > > >     > >>> protocols.
> > > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > > pre-defined
> > > > >     > >>> member
> > > > >     > >>>>> id.
> > > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> > optional
> > > > >     > >>>>>>>>> "group-change-timeout" value.
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the
> two
> > > > protocols
> > > > >     > >>> for
> > > > >     > >>>>> all
> > > > >     > >>>>>>>>> group members: we accept the protocol on the first
> > > joined
> > > > >     > >> member
> > > > >     > >>> of
> > > > >     > >>>>> the
> > > > >     > >>>>>>>>> group, and if later joining members indicate a
> > > different
> > > > >     > >>> membership
> > > > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > > > value was
> > > > >     > >>>>> different
> > > > >     > >>>>>>>> to
> > > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > > application that
> > > > >     > >>>> wants
> > > > >     > >>>>>> to
> > > > >     > >>>>>>>> switch to using static membership? Let's say there
> are
> > > 10
> > > > >     > >>> instances
> > > > >     > >>>> of
> > > > >     > >>>>>> it.
> > > > >     > >>>>>>>> As the instances go through a rolling restart, they
> > will
> > > > switch
> > > > >     > >>> from
> > > > >     > >>>>>>>> dynamic membership (the default?) to static
> > membership.
> > > > As each
> > > > >     > >>> one
> > > > >     > >>>>>> leaves
> > > > >     > >>>>>>>> the group and restarts, they will be rejected from
> the
> > > > group
> > > > >     > >>>> (because
> > > > >     > >>>>>> the
> > > > >     > >>>>>>>> group is currently using dynamic membership). The
> > group
> > > > will
> > > > >     > >>> shrink
> > > > >     > >>>>> down
> > > > >     > >>>>>>>> until there is 1 node handling all the traffic.
> After
> > > > that one
> > > > >     > >>>>> restarts,
> > > > >     > >>>>>>>> the group will switch over to static membership.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> Is that right? That means that the transition plan
> > from
> > > > dynamic
> > > > >     > >> to
> > > > >     > >>>>>> static
> > > > >     > >>>>>>>> membership isn't very smooth.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> > This
> > > > reminds
> > > > >     > >> me
> > > > >     > >>>> of
> > > > >     > >>>>>> the
> > > > >     > >>>>>>>> transition plans that were discussed for moving from
> > > > >     > >>> zookeeper-based
> > > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That
> > was
> > > > also
> > > > >     > >>> hard,
> > > > >     > >>>>> and
> > > > >     > >>>>>>>> ultimately we decided not to build that.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> -James
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> --
> > > > >     > >>>> -- Guozhang
> > > > >     > >>>>
> > > > >     > >>>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> --
> > > > >     > >> -- Guozhang
> > > > >     > >>
> > > > >     >
> > > > >
> > > > >
> > > >
> > > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Jason Gustafson <ja...@confluent.io>.
>
> I feel this would make the current protocol harder to use. For example, on
> KStream we typically would expect (number of instances * number of threads
> per instance) consumers. Giving out all this many member names in a list
> may not be easy, compared with dynamic generation of member names, at least
> for KStream use case.


That's a fair point. What I had in mind is for the API to specify the
number of consumers in the group. Then the ids would be determined using
some convention. Most likely, we would just use sequential numbers 0, 1, 2,
etc. We do the same thing for partition ids. Streams folks can chime in and
say whether that would work or not.

So this suggests we will not rely on heartbeat and commit interval to
> trigger rebalance? Even in static membership, I feel tracking active
> members is still required to be handled by broker since not all users are
> fully equipped with monitoring tools, otherwise users will feel
> uncomfortable using static membership solely for reducing rebalance purpose.


Let me clarify. I think it is still useful to distinguish the liveness of
the consumer using the heartbeat mechanism and to propagate that
information to the leader during rebalances. This gives the group leader
the option (but not the requirement) to change the partition assignment for
inactive members. My suggestion was more about how long the static
registration would remain valid. In the current proposal we have a
registration timeout, and I think Mayuresh is fair to point out the
potential confusion with the session timeout. It's kind of the same, but
not exactly. So instead, I was thinking that the registration is specified
ahead of time and remains valid until changed. It would be more like a
replica assignment. We don't move partitions just because a broker is down.
The expectation is that it the broker will eventually return. Similarly for
stateful applications, we would have the option to do the same thing. The
registration is fixed until someone changes it.

Does that make sense?

Thanks,
Jason


On Wed, Nov 14, 2018 at 2:46 PM, Boyang Chen <bc...@outlook.com> wrote:

> Thanks Jason for the suggestions! I update the KIP with full schema
> changes.
>
> > we offer an admin API that lets a user define the expected members of
> the group.
>
> I feel this would make the current protocol harder to use. For example, on
> KStream we typically would expect (number of instances * number of threads
> per instance) consumers. Giving out all this many member names in a list
> may not be easy, compared with dynamic generation of member names, at least
> for KStream use case.
>
> > For the sake of discussion, I was wondering if we could just say that
> static members do not expire.
>
>
> So this suggests we will not rely on heartbeat and commit interval to
> trigger rebalance? Even in static membership, I feel tracking active
> members is still required to be handled by broker since not all users are
> fully equipped with monitoring tools, otherwise users will feel
> uncomfortable using static membership solely for reducing rebalance purpose.
>
>
> For the admin API design, I'm simplifying the join group request handling,
> while using admin tool to switch between static and dynamic membership and
> set the two corresponding timeouts. Do you think this approach makes sense?
> The version one implementation will be much more clean if we handle
> membership change through user intervention.
>
>
> Best,
>
> Boyang
>
> ________________________________
> From: Jason Gustafson <ja...@confluent.io>
> Sent: Wednesday, November 14, 2018 9:31 AM
> To: dev
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey Boyang,
>
> Thanks for the updates. From a high level, I think this actually
> complements Konstantine's writeup on incremental rebalancing. The gap we're
> addressing is providing a way to bind the the partition assignment of a
> group to a set of user-provided ids so that we are not so reliant on the
> group's immediate state. For example, these ids might identify the state
> store volume for particular streams instances. This is basically what you
> need to work well with k8s stateful sets (as far as I understand them).
>
> One key decision is how we would define and update the expected static
> members in a consumer group. The mechanics of the registration and
> expansion timeouts feel a little bit clunky. For the sake of discussion, I
> was wondering if we could just say that static members do not expire.
> Instead, we offer an admin API that lets a user define the expected members
> of the group. This API could be used to both grow and shrink a group. This
> would solve the rebalancing problems when applications are initially
> bootstrapped or when they are restarted because we would always know how
> many members should be in a group. What do you think?
>
> By the way, it would be helpful to include the full schema definition for
> any protocol changes in the proposal.
>
> Thanks,
> Jason
>
>
> On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com> wrote:
>
> > Thanks Mayuresh for the feedback! Do you have a quick example for passing
> > in consumer config dynamically? I mainly use Kafka Streams at my daily
> work
> > so probably missing the idea how to do it in the current consumer
> setting.
> >
> >
> > For differentiating session timeout and registration timeout, I would try
> > to enhance the documentation in the first stage to see how people react
> to
> > the confusion (would be great if they feel straightforward!). Since one
> > doesn't have to fully understand the difference unless defining the new
> > config "member name", for current users we could buy some time to listen
> to
> > their understandings and improve our documentation correspondingly in the
> > follow-up KIPs.
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Mayuresh Gharat <gh...@gmail.com>
> > Sent: Sunday, November 11, 2018 1:06 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the reply.
> >
> > Please find the replies inline below :
> > For having a consumer config at runtime, I think it's not necessary to
> > address in this KIP because most companies run sidecar jobs through
> daemon
> > software like puppet. It should be easy to change the config through
> script
> > or UI without actual code change. We still want to leave flexibility for
> > user to define member name as they like.
> > ---- This might be little different for companies that use configuration
> > management tools that does not allow the applications to define/change
> the
> > configs dynamically. For example, if we use something similar to spring
> to
> > pull in the configs for the KafkaConsumer and pass it to the constructor
> to
> > create the KafkaConsumer object, it will be hard to specify a unique
> value
> > to the "MEMBER_NAME" config unless someone deploying the app generates a
> > unique string for this config outside the deployment workflow and copies
> it
> > statically before starting up each consumer instance. Unless we can
> loosen
> > the criteria for uniqueness of this config value, for each consumer
> > instance in the consumer group, I am not sure of a better way of
> > addressing this. If we don't want to loosen the criteria, then providing
> a
> > dynamic way to pass this in at runtime, would put the onus of having the
> > same unique value each time a consumer is restarted, on to the
> application
> > that is running the consumer.
> >
> > I just updated the kip about having both "registration timeout" and
> > "session timeout". The benefit of having two configs instead of one is to
> > reduce the mental burden for operation, for example user just needs to
> > unset "member name" to cast back to dynamic membership without worrying
> > about tuning the "session timeout" back to a smaller value.
> > --- That is a good point. I was thinking, if both the configs are
> > specified, it would be confusing for the end user without understanding
> the
> > internals of the consumer and its interaction with group coordinator, as
> > which takes precedence when and how it affects the consumer behavior.
> Just
> > my 2 cents.
> >
> > Thanks,
> >
> > Mayuresh
> >
> > On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > Hey Mayuresh,
> > >
> > >
> > > thanks for the thoughtful questions! Let me try to answer your
> questions
> > > one by one.
> > >
> > >
> > > For having a consumer config at runtime, I think it's not necessary to
> > > address in this KIP because most companies run sidecar jobs through
> > daemon
> > > software like puppet. It should be easy to change the config through
> > script
> > > or UI without actual code change. We still want to leave flexibility
> for
> > > user to define member name as they like.
> > >
> > >
> > > I just updated the kip about having both "registration timeout" and
> > > "session timeout". The benefit of having two configs instead of one is
> to
> > > reduce the mental burden for operation, for example user just needs to
> > > unset "member name" to cast back to dynamic membership without worrying
> > > about tuning the "session timeout" back to a smaller value.
> > >
> > >
> > > For backup topic, I think it's a low-level detail which could be
> > addressed
> > > in the implementation. I feel no preference of adding a new topic vs
> > reuse
> > > consumer offsets topic. I will do more analysis and make a trade-off
> > > comparison. Nice catch!
> > >
> > >
> > > I hope the explanations make sense to you. I will keep polishing on the
> > > edge cases and details.
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Mayuresh Gharat <gh...@gmail.com>
> > > Sent: Saturday, November 10, 2018 10:25 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hi Boyang,
> > >
> > > Thanks for the KIP and sorry for being late to the party. This KIP is
> > > really useful for us at Linkedin.
> > >
> > > I had a few questions :
> > >
> > > The idea of having static member name seems nice, but instead of a
> > config,
> > > would it be possible for it to be passed in to the consumer at runtime?
> > > This is because an app might want to decide the config value at runtime
> > > using its host information for example, to generate the unique member
> > name.
> > >
> > > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > > wondering if we can reuse the session timeout here. This might help us
> to
> > > have one less config on the consumer.
> > >
> > > The KIP also talks about adding another internal topic
> > "static_member_map".
> > > Would the semantics (GroupCoordinator broker, topic configs) be the
> same
> > as
> > > __consumer_offsets topic?
> > >
> > > Thanks,
> > >
> > > Mayuresh
> > >
> > >
> > > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com>
> wrote:
> > >
> > > > I took a quick pass of the proposal. First I would say it's a very
> > > > brilliant initiative from Konstantine and Confluent folks. To draft
> up
> > a
> > > > proposal like this needs deep understanding of the rebalance
> protocol!
> > I
> > > > summarized some thoughts here.
> > > >
> > > >
> > > > Overall the motivations of the two proposals align on that:
> > > >
> > > >   1.  Both believe the invariant resource (belonging to the same
> > process)
> > > > should be preserved across rebalance.
> > > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > > redistribution. I don't use rebalance here since part one of the
> > > > cooperative proposal could potentially introduce more rebalances but
> > only
> > > > on must-move resources.
> > > >   3.  Scale up/down and rolling bounce are causing unnecessary
> resource
> > > > shuffling that need to be mitigated.
> > > >
> > > >
> > > > On motivation level, I think both approach could solve/mitigate the
> > above
> > > > issues. They are just different in design philosophy, or I would say
> > the
> > > > perspective difference between framework user and algorithm designer.
> > > >
> > > >
> > > > Two proposals have different focuses. KIP-345 is trying to place more
> > > > fine-grained control on the broker side to reduce the unnecessary
> > > > rebalances, while keeping the client logic intact. This is pretty
> > > intuitive
> > > > cause-effect for normal developers who are not very familiar with
> > > rebalance
> > > > protocol. As a developer working with Kafka Streams daily, I'd be
> happy
> > > to
> > > > see a simplified rebalance protocol and just focus on maintaining the
> > > > stream/consumer jobs. Too many rebalances raised my concern on the
> job
> > > > health. To be concise, static membership has the advantage of
> reducing
> > > > mental burden.
> > > >
> > > >
> > > > Cooperative proposal takes thoughtful approach on client side. We
> want
> > to
> > > > have fine-grained control on the join/exit group behaviors and make
> the
> > > > current dynamic membership better to address above issues. I do feel
> > our
> > > > idea crossed on the delayed rebalance when we scale up/down, which
> > could
> > > > potentially reduce the state shuffling and decouple the behavior from
> > > > session timeout which is already overloaded.  In this sense, I
> believe
> > > both
> > > > approaches would serve well in making "reasonable rebalance" happen
> at
> > > the
> > > > "right timing".
> > > >
> > > >
> > > > However, based on my understanding, either 345 or cooperative
> > rebalancing
> > > > is not solving the problem Mike has proposed: could we do a better
> job
> > at
> > > > scaling up/down in ideal timing? My initial response was to introduce
> > an
> > > > admin API which now I feel is sub-optimal, in that the goal of smooth
> > > > transition is to make sure the newly up hosts are actually "ready".
> For
> > > > example:
> > > >
> > > >
> > > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At
> some
> > > > time we would like to scale up to 8 hosts, with the current
> > improvements
> > > we
> > > > could reduce 4 potential rebalances to a single one. But the new
> hosts
> > > are
> > > > yet unknown to be "ready" if they need to reconstruct the local
> state.
> > To
> > > > be actually ready, we need 4 standby tasks running on those empty
> hosts
> > > and
> > > > leader needs to wait for the signal of "replay/reconstruct complete"
> to
> > > > actually involve them into the main consumer group. Otherwise,
> > rebalance
> > > > just kills our performance since we need to wait indefinite long for
> > task
> > > > migration.
> > > >
> > > >
> > > > The scale down is also tricky such that we are not able to define a
> > > "true"
> > > > leave of a member. Rebalance immediately after "true" leaves are most
> > > > optimal comparing with human intervention. Does this make sense?
> > > >
> > > >
> > > > My intuition is that cooperative approach which was implemented on
> the
> > > > client side could better handle scaling cases than KIP 345, since it
> > > > involves a lot of algorithmic changes to define "replaying" stage,
> > which
> > > I
> > > > feel would over-complicate broker logic if implemented on
> coordinator.
> > If
> > > > we let 345 focus on reducing unnecessary rebalance, and let
> cooperative
> > > > approach focus on judging best timing of scale up/down, the two
> efforts
> > > > could be aligned. In long term, I feel the more complex improvement
> of
> > > > consumer protocol should happen on client side instead of server side
> > > which
> > > > is easier to test and has less global impact for the entire Kafka
> > > > production cluster.
> > > >
> > > >
> > > > Thanks again to Konstantine, Matthias and other folks in coming up
> with
> > > > this great client proposal. This is great complementation to KIP 345.
> > In
> > > a
> > > > high level, we are not having any collision on the path and both
> > > proposals
> > > > are making sense here. Just need better sync to avoid duplicate
> effort
> > :)
> > > >
> > > >
> > > > Best,
> > > >
> > > > Boyang
> > > >
> > > >
> > > > ________________________________
> > > > From: Boyang Chen <bc...@outlook.com>
> > > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > > > deeper look and make a comparison between the two proposals.
> > > >
> > > >
> > > > Meanwhile for the scale down specifically for stateful streaming, we
> > > could
> > > > actually introduce a new status called "learner" where the newly up
> > hosts
> > > > could try to catch up with the assigned task progress first before
> > > > triggering the rebalance, from which we don't see a sudden dip on the
> > > > progress. However, it is built on top of the success of KIP-345.
> > > >
> > > >
> > > > ________________________________
> > > > From: Matthias J. Sax <ma...@confluent.io>
> > > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > > specifying member id
> > > >
> > > > Hey,
> > > >
> > > > there was quite a pause on this KIP discussion and in the mean time,
> a
> > > > new design for incremental cooporative rebalance was suggested:
> > > >
> > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > Incremental Cooperative Rebalancing: Support and Policies ...<
> > https://cwiki.apache.org/confluence/display/KAFKA/Increm
> > ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> > cwiki.apache.org
> > Rebalancing between distributed application processes in Apache Kafka was
> > enhanced considerably when it was decoupled as logic from Kafka brokers
> and
> > was moved as responsibility to the clients and specifically to Kafka
> > Consumer. This pattern has been working robustly for quite a while now
> and
> > has ...
> >
> >
> >
> > > Incremental Cooperative Rebalancing: Support and Policies ...<
> > > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> > l+Cooperative+Rebalancing%3A+Support+and+Policies
> > > >
> > > cwiki.apache.org
> > > Rebalancing between distributed application processes in Apache Kafka
> was
> > > enhanced considerably when it was decoupled as logic from Kafka brokers
> > and
> > > was moved as responsibility to the clients and specifically to Kafka
> > > Consumer. This pattern has been working robustly for quite a while now
> > and
> > > has ...
> > >
> > >
> > >
> > > >
> > > >
> > > > We should make sure that the proposal and this KIP align to each
> other.
> > > > Thoughts?
> > > >
> > > >
> > > > -Matthias
> > > >
> > > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > > Hey Mike,
> > > > >
> > > > >
> > > > > thanks for the feedback, the two question are very thoughtful!
> > > > >
> > > > >
> > > > >> 1) I am a little confused about the distinction for the leader. If
> > the
> > > > consumer node that was assigned leader does a bounce (goes down and
> > > quickly
> > > > comes up) to update application code, will a rebalance be triggered?
> I
> > >
> > > do
> > > > not think a bounce of the leader should trigger a rebalance.
> > > > >
> > > > > For Q1 my intention was to minimize the change within one KIP,
> since
> > > the
> > > > leader rejoining case could be addressed separately.
> > > > >
> > > > >
> > > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > > gracefully increase the number of nodes in the cluster. I think we
> need
> > > to
> > > > support graceful shrink down as well. If I set the registration
> timeout
> > > to
> > > > 5 minutes > to handle rolling restarts or intermittent failures
> without
> > > > shuffling state, I don't want to wait 5 minutes in order for the
> group
> > to
> > > > rebalance if I am intentionally removing a node from the cluster. I
> am
> > > not
> > > > sure the best way to > do this. One idea I had was adding the ability
> > > for a
> > > > CLI or Admin API to force a rebalance of the group. This would allow
> > for
> > > an
> > > > admin to trigger the rebalance manually without waiting the entire
> > > > registration timeout on > shrink down. What do you think?
> > > > >
> > > > > For 2) my understanding is that for scaling down case it is better
> to
> > > be
> > > > addressed by CLI tool than code logic, since only by human evaluation
> > we
> > > > could decide whether it is a "right timing" -- the time when all the
> > > > scaling down consumers are offline -- to kick in rebalance. Unless we
> > > > introduce another term on coordinator which indicates the target
> > consumer
> > > > group size, broker will find it hard to decide when to start
> rebalance.
> > > So
> > > > far I prefer to hold the implementation for that, but agree we could
> > > > discuss whether we want to introduce admin API in this KIP or a
> > separate
> > > > one.
> > > > >
> > > > >
> > > > > Thanks again for the proposed ideas!
> > > > >
> > > > >
> > > > > Boyang
> > > > >
> > > > > ________________________________
> > > > > From: Mike Freyberger <mi...@xandr.com>
> > > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > > To: dev@kafka.apache.org
> > > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by
> > > > specifying member id
> > > > >
> > > > > Boyang,
> > > > >
> > > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > > >
> > > > > 1) I am a little confused about the distinction for the leader. If
> > the
> > > > consumer node that was assigned leader does a bounce (goes down and
> > > quickly
> > > > comes up) to update application code, will a rebalance be triggered?
> I
> > do
> > > > not think a bounce of the leader should trigger a rebalance.
> > > > >
> > > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > > gracefully increase the number of nodes in the cluster. I think we
> need
> > > to
> > > > support graceful shrink down as well. If I set the registration
> timeout
> > > to
> > > > 5 minutes to handle rolling restarts or intermittent failures without
> > > > shuffling state, I don't want to wait 5 minutes in order for the
> group
> > to
> > > > rebalance if I am intentionally removing a node from the cluster. I
> am
> > > not
> > > > sure the best way to do this. One idea I had was adding the ability
> > for a
> > > > CLI or Admin API to force a rebalance of the group. This would allow
> > for
> > > an
> > > > admin to trigger the rebalance manually without waiting the entire
> > > > registration timeout on shrink down. What do you think?
> > > > >
> > > > > Mike
> > > > >
> > > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > > > >
> > > > >     Btw, I updated KIP 345 based on my understanding. Feel free to
> > take
> > > > another round of look:
> > > > >
> > > > >
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > > KIP-345: Introduce static membership protocol to reduce ...<
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > >
> > > > > cwiki.apache.org
> > > > > For stateful applications, one of the biggest performance
> bottleneck
> > is
> > > > the state shuffling. In Kafka consumer, there is a concept called
> > > > "rebalance" which means that for given M partitions and N consumers
> in
> > > one
> > > > consumer group, Kafka will try to balance the load between consumers
> > and
> > > > ideally have ...
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > > >
> > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> > +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > >
> > > > >     cwiki.apache.org
> > > > >     For stateful applications, one of the biggest performance
> > > bottleneck
> > > > is the state shuffling. In Kafka consumer, there is a concept called
> > > > "rebalance" which means that for given M partitions and N consumers
> in
> > > one
> > > > consumer group, Kafka will try to balance the load between consumers
> > and
> > > > ideally have ...
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >     ________________________________
> > > > >     From: Boyang Chen <bc...@outlook.com>
> > > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > > >     To: dev@kafka.apache.org
> > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by specifying member id
> > > > >
> > > > >     Thanks everyone for the input on this thread! (Sorry it's been
> a
> > > > while) I feel that we are very close to the final solution.
> > > > >
> > > > >
> > > > >     Hey Jason and Mike, I have two quick questions on the new
> > features
> > > > here:
> > > > >
> > > > >       1.  so our proposal is that until we add a new static member
> > into
> > > > the group (scale up), we will not trigger rebalance until the
> > > "registration
> > > > timeout"( the member has been offline for too long)? How about
> leader's
> > > > rejoin request, I think we should still trigger rebalance when that
> > > > happens, since the consumer group may have new topics to consume?
> > > > >       2.  I'm not very clear on the scale up scenario in static
> > > > membership here. Should we fallback to dynamic membership while
> > > > adding/removing hosts (by setting member.name = null), or we still
> > want
> > > > to add instances with `member.name` so that we eventually
> > expand/shrink
> > > > the static membership? I personally feel the easier solution is to
> spin
> > > up
> > > > new members and wait until either the same "registration timeout" or
> a
> > > > "scale up timeout" before starting the rebalance. What do you think?
> > > > >
> > > > >     Meanwhile I will go ahead to make changes to the KIP with our
> > newly
> > > > discussed items and details. Really excited to see the design has
> > become
> > > > more solid.
> > > > >
> > > > >     Best,
> > > > >     Boyang
> > > > >
> > > > >     ________________________________
> > > > >     From: Jason Gustafson <ja...@confluent.io>
> > > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > > >     To: dev
> > > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > > > by specifying member id
> > > > >
> > > > >     Hey Mike,
> > > > >
> > > > >     Yeah, that's a good point. A long "registration timeout" may
> not
> > be
> > > > a great
> > > > >     idea. Perhaps in practice you'd set it long enough to be able
> to
> > > > detect a
> > > > >     failure and provision a new instance. Maybe on the order of 10
> > > > minutes is
> > > > >     more reasonable.
> > > > >
> > > > >     In any case, it's probably a good idea to have an
> administrative
> > > way
> > > > to
> > > > >     force deregistration. One option is to extend the DeleteGroups
> > API
> > > > with a
> > > > >     list of members names.
> > > > >
> > > > >     -Jason
> > > > >
> > > > >
> > > > >
> > > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > > mfreyberger@appnexus.com>
> > > > >     wrote:
> > > > >
> > > > >     > Jason,
> > > > >     >
> > > > >     > Regarding step 4 in your proposal which suggests beginning a
> > long
> > > > timer
> > > > >     > (30 minutes) when a static member leaves the group, would
> there
> > > > also be the
> > > > >     > ability for an admin to force a static membership expiration?
> > > > >     >
> > > > >     > I'm thinking that during particular types of outages or
> > upgrades
> > > > users
> > > > >     > would want forcefully remove a static member from the group.
> > > > >     >
> > > > >     > So the user would shut the consumer down normally, which
> > wouldn't
> > > > trigger
> > > > >     > a rebalance. Then the user could use an admin CLI tool to
> force
> > > > remove that
> > > > >     > consumer from the group, so the TopicPartitions that were
> > > > previously owned
> > > > >     > by that consumer can be released.
> > > > >     >
> > > > >     > At a high level, we need consumer groups to gracefully handle
> > > > intermittent
> > > > >     > failures and permanent failures. Currently, the consumer
> group
> > > > protocol
> > > > >     > handles permanent failures well, but does not handle
> > intermittent
> > > > failures
> > > > >     > well (it creates unnecessary rebalances). I want to make sure
> > the
> > > > overall
> > > > >     > solution here handles both intermittent failures and
> permanent
> > > > failures,
> > > > >     > rather than sacrificing support for permanent failures in
> order
> > > to
> > > > provide
> > > > >     > support for intermittent failures.
> > > > >     >
> > > > >     > Mike
> > > > >     >
> > > > >     > Sent from my iPhone
> > > > >     >
> > > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > > jason@confluent.io>
> > > > wrote:
> > > > >     > >
> > > > >     > > Hey Guozhang,
> > > > >     > >
> > > > >     > > Responses below:
> > > > >     > >
> > > > >     > > Originally I was trying to kill more birds with one stone
> > with
> > > > KIP-345,
> > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > shutting
> > > > down a
> > > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > > email), and
> > > > >     > >> hence proposing to have a pure static-membership protocol.
> > But
> > > > thinking
> > > > >     > >> twice about it I now feel it may be too ambitious and
> worth
> > > > fixing in
> > > > >     > >> another KIP.
> > > > >     > >
> > > > >     > >
> > > > >     > > I was considering an extension to support
> pre-initialization
> > of
> > > > the
> > > > >     > static
> > > > >     > > members of the group, but I agree we should probably leave
> > this
> > > > problem
> > > > >     > for
> > > > >     > > future work.
> > > > >     > >
> > > > >     > > 1. How this longish static member expiration timeout
> defined?
> > > Is
> > > > it via a
> > > > >     > >> broker, hence global config, or via a client config which
> > can
> > > be
> > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > >     > >
> > > > >     > >
> > > > >     > > I am not too sure. I tend to lean toward server-side
> configs
> > > > because they
> > > > >     > > are easier to evolve. If we have to add something to the
> > > > protocol, then
> > > > >     > > we'll be stuck with it forever.
> > > > >     > >
> > > > >     > > 2. Assuming that for static members, LEAVE_GROUP request
> will
> > > not
> > > > >     > trigger a
> > > > >     > >> rebalance immediately either, similar to session timeout,
> > but
> > > > only the
> > > > >     > >> longer member expiration timeout, can we remove the
> > internal "
> > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > walk-around
> > > > >     > then?
> > > > >     > >
> > > > >     > >
> > > > >     > > Yeah, I hope we can ultimately get rid of it, but we may
> need
> > > it
> > > > for
> > > > >     > > compatibility with older brokers. A related question is
> what
> > > > should be
> > > > >     > the
> > > > >     > > behavior of the consumer if `member.name` is provided but
> > the
> > > > broker
> > > > >     > does
> > > > >     > > not support it? We could either fail or silently downgrade
> to
> > > > dynamic
> > > > >     > > membership.
> > > > >     > >
> > > > >     > > -Jason
> > > > >     > >
> > > > >     > >
> > > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > >     > wrote:
> > > > >     > >>
> > > > >     > >> Hey Jason,
> > > > >     > >>
> > > > >     > >> I like your idea to simplify the upgrade protocol to allow
> > > > co-exist of
> > > > >     > >> static and dynamic members. Admittedly it may make the
> > > > coordinator-side
> > > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > > >     > >>
> > > > >     > >> Originally I was trying to kill more birds with one stone
> > with
> > > > KIP-345,
> > > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > > shutting
> > > > down a
> > > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > > email), and
> > > > >     > >> hence proposing to have a pure static-membership protocol.
> > But
> > > > thinking
> > > > >     > >> twice about it I now feel it may be too ambitious and
> worth
> > > > fixing in
> > > > >     > >> another KIP. With that, I think what you've proposed here
> > is a
> > > > good way
> > > > >     > to
> > > > >     > >> go for KIP-345 itself.
> > > > >     > >>
> > > > >     > >> Note there are a few details in your proposal we'd still
> > need
> > > > to figure
> > > > >     > >> out:
> > > > >     > >>
> > > > >     > >> 1. How this longish static member expiration timeout
> > defined?
> > > > Is it via
> > > > >     > a
> > > > >     > >> broker, hence global config, or via a client config which
> > can
> > > be
> > > > >     > >> communicated to broker via JoinGroupRequest?
> > > > >     > >>
> > > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> > will
> > > > not
> > > > >     > trigger a
> > > > >     > >> rebalance immediately either, similar to session timeout,
> > but
> > > > only the
> > > > >     > >> longer member expiration timeout, can we remove the
> > internal "
> > > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > > walk-around
> > > > >     > then?
> > > > >     > >>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> Guozhang
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > > jason@confluent.io>
> > > > >     > >> wrote:
> > > > >     > >>
> > > > >     > >>> Hey All,
> > > > >     > >>>
> > > > >     > >>> Nice to see some solid progress on this. It sounds like
> one
> > > of
> > > > the
> > > > >     > >>> complications is allowing static and dynamic registration
> > to
> > > > coexist.
> > > > >     > I'm
> > > > >     > >>> wondering if we can do something like the following:
> > > > >     > >>>
> > > > >     > >>> 1. Statically registered members (those joining the group
> > > with
> > > > a
> > > > >     > >> non-null `
> > > > >     > >>> member.name`) maintain a session with the coordinator
> just
> > > > like
> > > > >     > dynamic
> > > > >     > >>> members.
> > > > >     > >>> 2. If a session is active for a static member when a
> > > rebalance
> > > > begins,
> > > > >     > >> then
> > > > >     > >>> basically we'll keep the current behavior. The rebalance
> > will
> > > > await the
> > > > >     > >>> static member joining the group.
> > > > >     > >>> 3. If a static member does not have an active session,
> then
> > > the
> > > > >     > >> coordinator
> > > > >     > >>> will not wait for it to join, but will still include it
> in
> > > the
> > > > >     > rebalance.
> > > > >     > >>> The coordinator will forward the cached subscription
> > > > information to the
> > > > >     > >>> leader and will cache the assignment after the rebalance
> > > > completes.
> > > > >     > (Note
> > > > >     > >>> that we still have the generationId to fence offset
> commits
> > > > from a
> > > > >     > static
> > > > >     > >>> zombie if the assignment changes.)
> > > > >     > >>> 4. When a static member leaves the group or has its
> session
> > > > expire, no
> > > > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > > > expire the
> > > > >     > >> static
> > > > >     > >>> registration. This would be a longish timeout (like 30
> > > minutes
> > > > say).
> > > > >     > >>>
> > > > >     > >>> So basically static members participate in all rebalances
> > > > regardless
> > > > >     > >>> whether they have an active session. In a given
> rebalance,
> > > > some of the
> > > > >     > >>> members may be static and some dynamic. The group leader
> > can
> > > > >     > >> differentiate
> > > > >     > >>> the two based on the presence of the `member.name` (we
> > have
> > > > to add
> > > > >     > this
> > > > >     > >> to
> > > > >     > >>> the JoinGroupResponse). Generally speaking, we would
> choose
> > > > leaders
> > > > >     > >>> preferentially from the active members that support the
> > > latest
> > > > >     > JoinGroup
> > > > >     > >>> protocol and are using static membership. If we have to
> > > choose
> > > > a leader
> > > > >     > >>> with an old version, however, it would see all members in
> > the
> > > > group
> > > > >     > >> (static
> > > > >     > >>> or dynamic) as dynamic members and perform the assignment
> > as
> > > > usual.
> > > > >     > >>>
> > > > >     > >>> Would that work?
> > > > >     > >>>
> > > > >     > >>> -Jason
> > > > >     > >>>
> > > > >     > >>>
> > > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > > wangguoz@gmail.com>
> > > > >     > >> wrote:
> > > > >     > >>>
> > > > >     > >>>> Hello Boyang,
> > > > >     > >>>>
> > > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > > >     > >>>>
> > > > >     > >>>> 1. Where will "change-group-timeout" be communicated to
> > the
> > > > broker?
> > > > >     > >> Will
> > > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> > going
> > > > to
> > > > >     > >>> piggy-back
> > > > >     > >>>> on the existing session-timeout field (assuming that the
> > > > original
> > > > >     > value
> > > > >     > >>>> will not be used anywhere in the static membership any
> > > more)?
> > > > >     > >>>>
> > > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > > timeout to
> > > > >     > >> return,
> > > > >     > >>>> we shall still trigger rebalance but it could still try
> to
> > > > catch
> > > > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> > > your
> > > > proposal
> > > > >     > >> is
> > > > >     > >>>> that for static memberships, the broker will NOT trigger
> > > > rebalance
> > > > >     > even
> > > > >     > >>>> after session-timeout has been detected, but only that
> > after
> > > > >     > >>>> change-group-timeout
> > > > >     > >>>> which is supposed to be longer than session-timeout to
> be
> > > > defined?
> > > > >     > >>>>
> > > > >     > >>>> 3. "A join group request with member.name set will be
> > > > treated as
> > > > >     > >>>> `static-membership` strategy", in this case, how would
> the
> > > > switch from
> > > > >     > >>>> dynamic to static happen, since whoever changed the
> > > > member.name to
> > > > >     > >>>> not-null
> > > > >     > >>>> will be rejected, right?
> > > > >     > >>>>
> > > > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > > > timeout to
> > > > >     > >>> trigger
> > > > >     > >>>> rebalance should be sufficient." this is also a bit
> > unclear
> > > > to me: who
> > > > >     > >>> will
> > > > >     > >>>> erase the cached mapping? Since it is on the
> broker-side I
> > > > assume that
> > > > >     > >>>> broker has to do it. Are you suggesting to use a new
> > request
> > > > for it?
> > > > >     > >>>>
> > > > >     > >>>> 5. "Halfway switch": following 3) above, if your
> proposal
> > is
> > > > basically
> > > > >     > >> to
> > > > >     > >>>> let "first join-request wins", and the strategy will
> stay
> > as
> > > > is until
> > > > >     > >> all
> > > > >     > >>>> members are gone, then this will also not happen since
> > > > whoever used
> > > > >     > >>>> different strategy as the first guy who sends join-group
> > > > request will
> > > > >     > >> be
> > > > >     > >>>> rejected right?
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> Guozhang
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > > john@confluent.io>
> > > > >     > >> wrote:
> > > > >     > >>>>
> > > > >     > >>>>> This sounds good to me!
> > > > >     > >>>>>
> > > > >     > >>>>> Thanks for the time you've spent on it,
> > > > >     > >>>>> -John
> > > > >     > >>>>>
> > > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > > bchen11@outlook.com>
> > > > >     > >>>> wrote:
> > > > >     > >>>>>
> > > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy
> recently
> > > and
> > > > >     > >> haven't
> > > > >     > >>>> got
> > > > >     > >>>>>> time to update this thread. To summarize what we come
> up
> > > so
> > > > far,
> > > > >     > >> here
> > > > >     > >>>> is
> > > > >     > >>>>> a
> > > > >     > >>>>>> draft updated plan:
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Introduce a new config called `member.name` which is
> > > > supposed to
> > > > >     > >> be
> > > > >     > >>>>>> provided uniquely by the consumer client. The broker
> > will
> > > > maintain
> > > > >     > >> a
> > > > >     > >>>>> cache
> > > > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > > > request with
> > > > >     > >>>>>> member.name set will be treated as
> `static-membership`
> > > > strategy,
> > > > >     > >> and
> > > > >     > >>>>> will
> > > > >     > >>>>>> reject any join group request without member.name. So
> > > this
> > > > >     > >>>> coordination
> > > > >     > >>>>>> change will be differentiated from the
> > > `dynamic-membership`
> > > > >     > >> protocol
> > > > >     > >>> we
> > > > >     > >>>>>> currently have.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When handling static join group request:
> > > > >     > >>>>>>
> > > > >     > >>>>>>  1.   The broker will check the membership to see
> > whether
> > > > this is
> > > > >     > >> a
> > > > >     > >>>> new
> > > > >     > >>>>>> member. If new, broker allocate a unique member id,
> > cache
> > > > the
> > > > >     > >> mapping
> > > > >     > >>>> and
> > > > >     > >>>>>> move to rebalance stage.
> > > > >     > >>>>>>  2.   Following 1, if this is an existing member,
> broker
> > > > will not
> > > > >     > >>>> change
> > > > >     > >>>>>> group state, and return its cached member.id and
> > current
> > > > >     > >> assignment.
> > > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin
> > with
> > > > pair
> > > > >     > >>> member
> > > > >     > >>>>>> name and id, I think for join group request it is ok
> to
> > > > leave
> > > > >     > >> member
> > > > >     > >>> id
> > > > >     > >>>>>> blank as member name is the unique identifier. In
> commit
> > > > offset
> > > > >     > >>> request
> > > > >     > >>>>> we
> > > > >     > >>>>>> *must* have both.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When handling commit offset request, if enabled with
> > > static
> > > > >     > >>> membership,
> > > > >     > >>>>>> each time the commit request must have both
> member.name
> > > and
> > > > >     > >>> member.id
> > > > >     > >>>> to
> > > > >     > >>>>>> be identified as a `certificated member`. If not, this
> > > > means there
> > > > >     > >>> are
> > > > >     > >>>>>> duplicate consumer members with same member name and
> the
> > > > request
> > > > >     > >> will
> > > > >     > >>>> be
> > > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> > client
> > > > will
> > > > >     > >> send a
> > > > >     > >>>>>> leave group request (static membership mode). In
> static
> > > > membership,
> > > > >     > >>> we
> > > > >     > >>>>> will
> > > > >     > >>>>>> also define `change-group-timeout` to hold on
> rebalance
> > > > provided by
> > > > >     > >>>>> leader.
> > > > >     > >>>>>> So we will wait for all the members to rejoin the
> group
> > > and
> > > > do
> > > > >     > >>> exactly
> > > > >     > >>>>> one
> > > > >     > >>>>>> rebalance since all members are expected to rejoin
> > within
> > > > timeout.
> > > > >     > >> If
> > > > >     > >>>>>> consumer crashes, the join group request from the
> > > restarted
> > > > >     > >> consumer
> > > > >     > >>>> will
> > > > >     > >>>>>> be recognized as an existing member and be handled as
> > > above
> > > > >     > >> condition
> > > > >     > >>>> 1;
> > > > >     > >>>>>> However, if the consumer takes longer than session
> > timeout
> > > > to
> > > > >     > >> return,
> > > > >     > >>>> we
> > > > >     > >>>>>> shall still trigger rebalance but it could still try
> to
> > > > catch
> > > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > > timeout, its
> > > > >     > >>>> cached
> > > > >     > >>>>>> state on broker will be garbage collected and trigger
> a
> > > new
> > > > >     > >> rebalance
> > > > >     > >>>>> when
> > > > >     > >>>>>> it finally joins.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> And consider the switch between dynamic to static
> > > > membership.
> > > > >     > >>>>>>
> > > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise
> > the
> > > > >     > >> membership
> > > > >     > >>>> to
> > > > >     > >>>>>> static and wait for all the current members to
> restart,
> > > > since their
> > > > >     > >>>>>> membership is still dynamic. Here our assumption is
> that
> > > the
> > > > >     > >> restart
> > > > >     > >>>>>> process shouldn't take a long time, as long restart is
> > > > breaking the
> > > > >     > >>>>>> `rebalance timeout` in whatever membership protocol we
> > are
> > > > using.
> > > > >     > >>>> Before
> > > > >     > >>>>>> restart, all dynamic member join requests will be
> > > rejected.
> > > > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> > > which
> > > > should
> > > > >     > >>> be
> > > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > > session
> > > > timeout
> > > > >     > >>> to
> > > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > > current
> > > > >     > >>> behavior)
> > > > >     > >>>>>>  3.  Halfway switch: a corner case is like some
> clients
> > > keep
> > > > >     > >> dynamic
> > > > >     > >>>>>> membership while some keep static membership. This
> will
> > > > cause the
> > > > >     > >>> group
> > > > >     > >>>>>> rebalance forever without progress because
> > dynamic/static
> > > > states
> > > > >     > >> are
> > > > >     > >>>>>> bouncing each other. This could guarantee that we will
> > not
> > > > make the
> > > > >     > >>>>>> consumer group work in a wrong state by having half
> > static
> > > > and half
> > > > >     > >>>>> dynamic.
> > > > >     > >>>>>>
> > > > >     > >>>>>> To guarantee correctness, we will also push the member
> > > > name/id pair
> > > > >     > >>> to
> > > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > > > upgrade the
> > > > >     > >> API
> > > > >     > >>>>>> version, these details will be further discussed back
> in
> > > > the KIP.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Are there any concern for this high level proposal?
> Just
> > > > want to
> > > > >     > >>>>> reiterate
> > > > >     > >>>>>> on the core idea of the KIP: "If the broker recognize
> > this
> > > > consumer
> > > > >     > >>> as
> > > > >     > >>>> an
> > > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > > >     > >>>>>>
> > > > >     > >>>>>> Thanks a lot for everyone's input! I feel this
> proposal
> > is
> > > > much
> > > > >     > >> more
> > > > >     > >>>>>> robust than previous one!
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> Best,
> > > > >     > >>>>>>
> > > > >     > >>>>>> Boyang
> > > > >     > >>>>>>
> > > > >     > >>>>>> ________________________________
> > > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > > >     > >>>>>> To: dev@kafka.apache.org
> > > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple
> consumer
> > > > rebalances
> > > > >     > >>> by
> > > > >     > >>>>>> specifying member id
> > > > >     > >>>>>>
> > > > >     > >>>>>> Hi,
> > > > >     > >>>>>>
> > > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> > about
> > > > internals
> > > > >     > >>>> again
> > > > >     > >>>>>> :)
> > > > >     > >>>>>>
> > > > >     > >>>>>> I like the idea or a user config `member.name` and to
> > > keep
> > > > `
> > > > >     > >>> member.id`
> > > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > > client.id`
> > > > might
> > > > >     > >>> not
> > > > >     > >>>>>> be a good idea.
> > > > >     > >>>>>>
> > > > >     > >>>>>> To clarify the algorithm, each time we generate a new
> `
> > > > member.id`,
> > > > >     > >>> we
> > > > >     > >>>>>> also need to update the "group membership" information
> > > (ie,
> > > > mapping
> > > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `
> member.id
> > `
> > > > replaces
> > > > >     > >>> the
> > > > >     > >>>>>> old entry in the cache.
> > > > >     > >>>>>>
> > > > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > > > member.id`
> > > > >     > >>>> mapping
> > > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should
> mention
> > > > this IMHO.
> > > > >     > >>>>>>
> > > > >     > >>>>>> For changing the default value of config
> > > > `leave.group.on.close`. I
> > > > >     > >>>> agree
> > > > >     > >>>>>> with John, that we should not change the default
> config,
> > > > because it
> > > > >     > >>>>>> would impact all consumer groups with dynamic
> > assignment.
> > > > However,
> > > > >     > >> I
> > > > >     > >>>>>> think we can document, that if static assignment is
> used
> > > > (ie,
> > > > >     > >>>>>> `member.name` is configured) we never send a
> > > > LeaveGroupRequest
> > > > >     > >>>>>> regardless of the config. Note, that the config is
> > > > internal, so not
> > > > >     > >>>> sure
> > > > >     > >>>>>> how to document this in detail. We should not expose
> the
> > > > internal
> > > > >     > >>>> config
> > > > >     > >>>>>> in the docs.
> > > > >     > >>>>>>
> > > > >     > >>>>>> About upgrading: why do we need have two rolling
> bounces
> > > > and encode
> > > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > > >     > >>>>>>
> > > > >     > >>>>>> If we upgrade an existing consumer group from dynamic
> to
> > > > static, I
> > > > >     > >>>> don't
> > > > >     > >>>>>> see any reason why both should not work together and
> > > single
> > > > rolling
> > > > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > > > consumer and
> > > > >     > >>>>>> switch from dynamic to static, it sends a `
> member.name`
> > > > and the
> > > > >     > >>> broker
> > > > >     > >>>>>> registers the [member.name, member.id] in the cache.
> > Why
> > > > would
> > > > >     > >> this
> > > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > > assignment?
> > > > >     > >>>>>>
> > > > >     > >>>>>> Also, Guozhang mentioned that for all other request,
> we
> > > > need to
> > > > >     > >> check
> > > > >     > >>>> if
> > > > >     > >>>>>> the mapping [member.name, member.id] contains the
> send
> > `
> > > > member.id`
> > > > >     > >>> --
> > > > >     > >>>> I
> > > > >     > >>>>>> don't think this is necessary -- it seems to be
> > sufficient
> > > > to check
> > > > >     > >>> the
> > > > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping
> as
> > > be
> > > > do
> > > > >     > >> today
> > > > >     > >>> --
> > > > >     > >>>>>> thus, checking `member.id` does not require any
> change
> > > > IMHO.
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>> -Matthias
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > > >     > >>>>>>> @James
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> What you described is true: the transition from
> dynamic
> > > to
> > > > static
> > > > >     > >>>>>>> memberships are not thought through yet. But I do not
> > > > think it is
> > > > >     > >>> an
> > > > >     > >>>>>>> impossible problem: note that we indeed moved the
> > offset
> > > > commit
> > > > >     > >>> from
> > > > >     > >>>> ZK
> > > > >     > >>>>>> to
> > > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is
> to
> > > > first to
> > > > >     > >>>>>>> double-commits on both zk and coordinator, and then
> do
> > a
> > > > second
> > > > >     > >>> round
> > > > >     > >>>>> to
> > > > >     > >>>>>>> turn the zk off.
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > > >     > >>>> two-rolling-bounce
> > > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag
> to
> > > > "static"
> > > > >     > >>> while
> > > > >     > >>>>>> keep
> > > > >     > >>>>>>> the registry-id field empty still, in this case, the
> > > > coordinator
> > > > >     > >>>> still
> > > > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> > > while
> > > > >     > >>> allowing
> > > > >     > >>>>> the
> > > > >     > >>>>>>> protocol to be set to "static"; after the first
> rolling
> > > > bounce,
> > > > >     > >> the
> > > > >     > >>>>> group
> > > > >     > >>>>>>> protocol is already "static", then a second rolling
> > > bounce
> > > > is
> > > > >     > >>>> triggered
> > > > >     > >>>>>> and
> > > > >     > >>>>>>> this time we set the registry-id.
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> Guozhang
> > > > >     > >>>>>>>
> > > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > > >     > >> wushujames@gmail.com>
> > > > >     > >>>>>> wrote:
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> > this:
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > > wangguoz@gmail.com
> > > > >     > >>>
> > > > >     > >>>>> wrote:
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > > > fields:
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> > membership
> > > > >     > >>> protocols.
> > > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > > pre-defined
> > > > >     > >>> member
> > > > >     > >>>>> id.
> > > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> > optional
> > > > >     > >>>>>>>>> "group-change-timeout" value.
> > > > >     > >>>>>>>>>
> > > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the
> two
> > > > protocols
> > > > >     > >>> for
> > > > >     > >>>>> all
> > > > >     > >>>>>>>>> group members: we accept the protocol on the first
> > > joined
> > > > >     > >> member
> > > > >     > >>> of
> > > > >     > >>>>> the
> > > > >     > >>>>>>>>> group, and if later joining members indicate a
> > > different
> > > > >     > >>> membership
> > > > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > > > value was
> > > > >     > >>>>> different
> > > > >     > >>>>>>>> to
> > > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > > application that
> > > > >     > >>>> wants
> > > > >     > >>>>>> to
> > > > >     > >>>>>>>> switch to using static membership? Let's say there
> are
> > > 10
> > > > >     > >>> instances
> > > > >     > >>>> of
> > > > >     > >>>>>> it.
> > > > >     > >>>>>>>> As the instances go through a rolling restart, they
> > will
> > > > switch
> > > > >     > >>> from
> > > > >     > >>>>>>>> dynamic membership (the default?) to static
> > membership.
> > > > As each
> > > > >     > >>> one
> > > > >     > >>>>>> leaves
> > > > >     > >>>>>>>> the group and restarts, they will be rejected from
> the
> > > > group
> > > > >     > >>>> (because
> > > > >     > >>>>>> the
> > > > >     > >>>>>>>> group is currently using dynamic membership). The
> > group
> > > > will
> > > > >     > >>> shrink
> > > > >     > >>>>> down
> > > > >     > >>>>>>>> until there is 1 node handling all the traffic.
> After
> > > > that one
> > > > >     > >>>>> restarts,
> > > > >     > >>>>>>>> the group will switch over to static membership.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> Is that right? That means that the transition plan
> > from
> > > > dynamic
> > > > >     > >> to
> > > > >     > >>>>>> static
> > > > >     > >>>>>>>> membership isn't very smooth.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> > This
> > > > reminds
> > > > >     > >> me
> > > > >     > >>>> of
> > > > >     > >>>>>> the
> > > > >     > >>>>>>>> transition plans that were discussed for moving from
> > > > >     > >>> zookeeper-based
> > > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That
> > was
> > > > also
> > > > >     > >>> hard,
> > > > >     > >>>>> and
> > > > >     > >>>>>>>> ultimately we decided not to build that.
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>> -James
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>>
> > > > >     > >>>>>
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>>
> > > > >     > >>>> --
> > > > >     > >>>> -- Guozhang
> > > > >     > >>>>
> > > > >     > >>>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >>
> > > > >     > >> --
> > > > >     > >> -- Guozhang
> > > > >     > >>
> > > > >     >
> > > > >
> > > > >
> > > >
> > > >
> > >
> > > --
> > > -Regards,
> > > Mayuresh R. Gharat
> > > (862) 250-7125
> > >
> >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Jason for the suggestions! I update the KIP with full schema changes.

> we offer an admin API that lets a user define the expected members of the group.

I feel this would make the current protocol harder to use. For example, on KStream we typically would expect (number of instances * number of threads per instance) consumers. Giving out all this many member names in a list may not be easy, compared with dynamic generation of member names, at least for KStream use case.

> For the sake of discussion, I was wondering if we could just say that static members do not expire.


So this suggests we will not rely on heartbeat and commit interval to trigger rebalance? Even in static membership, I feel tracking active members is still required to be handled by broker since not all users are fully equipped with monitoring tools, otherwise users will feel uncomfortable using static membership solely for reducing rebalance purpose.


For the admin API design, I'm simplifying the join group request handling, while using admin tool to switch between static and dynamic membership and set the two corresponding timeouts. Do you think this approach makes sense? The version one implementation will be much more clean if we handle membership change through user intervention.


Best,

Boyang

________________________________
From: Jason Gustafson <ja...@confluent.io>
Sent: Wednesday, November 14, 2018 9:31 AM
To: dev
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hey Boyang,

Thanks for the updates. From a high level, I think this actually
complements Konstantine's writeup on incremental rebalancing. The gap we're
addressing is providing a way to bind the the partition assignment of a
group to a set of user-provided ids so that we are not so reliant on the
group's immediate state. For example, these ids might identify the state
store volume for particular streams instances. This is basically what you
need to work well with k8s stateful sets (as far as I understand them).

One key decision is how we would define and update the expected static
members in a consumer group. The mechanics of the registration and
expansion timeouts feel a little bit clunky. For the sake of discussion, I
was wondering if we could just say that static members do not expire.
Instead, we offer an admin API that lets a user define the expected members
of the group. This API could be used to both grow and shrink a group. This
would solve the rebalancing problems when applications are initially
bootstrapped or when they are restarted because we would always know how
many members should be in a group. What do you think?

By the way, it would be helpful to include the full schema definition for
any protocol changes in the proposal.

Thanks,
Jason


On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com> wrote:

> Thanks Mayuresh for the feedback! Do you have a quick example for passing
> in consumer config dynamically? I mainly use Kafka Streams at my daily work
> so probably missing the idea how to do it in the current consumer setting.
>
>
> For differentiating session timeout and registration timeout, I would try
> to enhance the documentation in the first stage to see how people react to
> the confusion (would be great if they feel straightforward!). Since one
> doesn't have to fully understand the difference unless defining the new
> config "member name", for current users we could buy some time to listen to
> their understandings and improve our documentation correspondingly in the
> follow-up KIPs.
>
>
> Boyang
>
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Sunday, November 11, 2018 1:06 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for the reply.
>
> Please find the replies inline below :
> For having a consumer config at runtime, I think it's not necessary to
> address in this KIP because most companies run sidecar jobs through daemon
> software like puppet. It should be easy to change the config through script
> or UI without actual code change. We still want to leave flexibility for
> user to define member name as they like.
> ---- This might be little different for companies that use configuration
> management tools that does not allow the applications to define/change the
> configs dynamically. For example, if we use something similar to spring to
> pull in the configs for the KafkaConsumer and pass it to the constructor to
> create the KafkaConsumer object, it will be hard to specify a unique value
> to the "MEMBER_NAME" config unless someone deploying the app generates a
> unique string for this config outside the deployment workflow and copies it
> statically before starting up each consumer instance. Unless we can loosen
> the criteria for uniqueness of this config value, for each consumer
> instance in the consumer group, I am not sure of a better way of
> addressing this. If we don't want to loosen the criteria, then providing a
> dynamic way to pass this in at runtime, would put the onus of having the
> same unique value each time a consumer is restarted, on to the application
> that is running the consumer.
>
> I just updated the kip about having both "registration timeout" and
> "session timeout". The benefit of having two configs instead of one is to
> reduce the mental burden for operation, for example user just needs to
> unset "member name" to cast back to dynamic membership without worrying
> about tuning the "session timeout" back to a smaller value.
> --- That is a good point. I was thinking, if both the configs are
> specified, it would be confusing for the end user without understanding the
> internals of the consumer and its interaction with group coordinator, as
> which takes precedence when and how it affects the consumer behavior. Just
> my 2 cents.
>
> Thanks,
>
> Mayuresh
>
> On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:
>
> > Hey Mayuresh,
> >
> >
> > thanks for the thoughtful questions! Let me try to answer your questions
> > one by one.
> >
> >
> > For having a consumer config at runtime, I think it's not necessary to
> > address in this KIP because most companies run sidecar jobs through
> daemon
> > software like puppet. It should be easy to change the config through
> script
> > or UI without actual code change. We still want to leave flexibility for
> > user to define member name as they like.
> >
> >
> > I just updated the kip about having both "registration timeout" and
> > "session timeout". The benefit of having two configs instead of one is to
> > reduce the mental burden for operation, for example user just needs to
> > unset "member name" to cast back to dynamic membership without worrying
> > about tuning the "session timeout" back to a smaller value.
> >
> >
> > For backup topic, I think it's a low-level detail which could be
> addressed
> > in the implementation. I feel no preference of adding a new topic vs
> reuse
> > consumer offsets topic. I will do more analysis and make a trade-off
> > comparison. Nice catch!
> >
> >
> > I hope the explanations make sense to you. I will keep polishing on the
> > edge cases and details.
> >
> >
> > Best,
> >
> > Boyang
> >
> > ________________________________
> > From: Mayuresh Gharat <gh...@gmail.com>
> > Sent: Saturday, November 10, 2018 10:25 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the KIP and sorry for being late to the party. This KIP is
> > really useful for us at Linkedin.
> >
> > I had a few questions :
> >
> > The idea of having static member name seems nice, but instead of a
> config,
> > would it be possible for it to be passed in to the consumer at runtime?
> > This is because an app might want to decide the config value at runtime
> > using its host information for example, to generate the unique member
> name.
> >
> > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > wondering if we can reuse the session timeout here. This might help us to
> > have one less config on the consumer.
> >
> > The KIP also talks about adding another internal topic
> "static_member_map".
> > Would the semantics (GroupCoordinator broker, topic configs) be the same
> as
> > __consumer_offsets topic?
> >
> > Thanks,
> >
> > Mayuresh
> >
> >
> > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > I took a quick pass of the proposal. First I would say it's a very
> > > brilliant initiative from Konstantine and Confluent folks. To draft up
> a
> > > proposal like this needs deep understanding of the rebalance protocol!
> I
> > > summarized some thoughts here.
> > >
> > >
> > > Overall the motivations of the two proposals align on that:
> > >
> > >   1.  Both believe the invariant resource (belonging to the same
> process)
> > > should be preserved across rebalance.
> > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > redistribution. I don't use rebalance here since part one of the
> > > cooperative proposal could potentially introduce more rebalances but
> only
> > > on must-move resources.
> > >   3.  Scale up/down and rolling bounce are causing unnecessary resource
> > > shuffling that need to be mitigated.
> > >
> > >
> > > On motivation level, I think both approach could solve/mitigate the
> above
> > > issues. They are just different in design philosophy, or I would say
> the
> > > perspective difference between framework user and algorithm designer.
> > >
> > >
> > > Two proposals have different focuses. KIP-345 is trying to place more
> > > fine-grained control on the broker side to reduce the unnecessary
> > > rebalances, while keeping the client logic intact. This is pretty
> > intuitive
> > > cause-effect for normal developers who are not very familiar with
> > rebalance
> > > protocol. As a developer working with Kafka Streams daily, I'd be happy
> > to
> > > see a simplified rebalance protocol and just focus on maintaining the
> > > stream/consumer jobs. Too many rebalances raised my concern on the job
> > > health. To be concise, static membership has the advantage of reducing
> > > mental burden.
> > >
> > >
> > > Cooperative proposal takes thoughtful approach on client side. We want
> to
> > > have fine-grained control on the join/exit group behaviors and make the
> > > current dynamic membership better to address above issues. I do feel
> our
> > > idea crossed on the delayed rebalance when we scale up/down, which
> could
> > > potentially reduce the state shuffling and decouple the behavior from
> > > session timeout which is already overloaded.  In this sense, I believe
> > both
> > > approaches would serve well in making "reasonable rebalance" happen at
> > the
> > > "right timing".
> > >
> > >
> > > However, based on my understanding, either 345 or cooperative
> rebalancing
> > > is not solving the problem Mike has proposed: could we do a better job
> at
> > > scaling up/down in ideal timing? My initial response was to introduce
> an
> > > admin API which now I feel is sub-optimal, in that the goal of smooth
> > > transition is to make sure the newly up hosts are actually "ready". For
> > > example:
> > >
> > >
> > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> > > time we would like to scale up to 8 hosts, with the current
> improvements
> > we
> > > could reduce 4 potential rebalances to a single one. But the new hosts
> > are
> > > yet unknown to be "ready" if they need to reconstruct the local state.
> To
> > > be actually ready, we need 4 standby tasks running on those empty hosts
> > and
> > > leader needs to wait for the signal of "replay/reconstruct complete" to
> > > actually involve them into the main consumer group. Otherwise,
> rebalance
> > > just kills our performance since we need to wait indefinite long for
> task
> > > migration.
> > >
> > >
> > > The scale down is also tricky such that we are not able to define a
> > "true"
> > > leave of a member. Rebalance immediately after "true" leaves are most
> > > optimal comparing with human intervention. Does this make sense?
> > >
> > >
> > > My intuition is that cooperative approach which was implemented on the
> > > client side could better handle scaling cases than KIP 345, since it
> > > involves a lot of algorithmic changes to define "replaying" stage,
> which
> > I
> > > feel would over-complicate broker logic if implemented on coordinator.
> If
> > > we let 345 focus on reducing unnecessary rebalance, and let cooperative
> > > approach focus on judging best timing of scale up/down, the two efforts
> > > could be aligned. In long term, I feel the more complex improvement of
> > > consumer protocol should happen on client side instead of server side
> > which
> > > is easier to test and has less global impact for the entire Kafka
> > > production cluster.
> > >
> > >
> > > Thanks again to Konstantine, Matthias and other folks in coming up with
> > > this great client proposal. This is great complementation to KIP 345.
> In
> > a
> > > high level, we are not having any collision on the path and both
> > proposals
> > > are making sense here. Just need better sync to avoid duplicate effort
> :)
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > >
> > > ________________________________
> > > From: Boyang Chen <bc...@outlook.com>
> > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > > deeper look and make a comparison between the two proposals.
> > >
> > >
> > > Meanwhile for the scale down specifically for stateful streaming, we
> > could
> > > actually introduce a new status called "learner" where the newly up
> hosts
> > > could try to catch up with the assigned task progress first before
> > > triggering the rebalance, from which we don't see a sudden dip on the
> > > progress. However, it is built on top of the success of KIP-345.
> > >
> > >
> > > ________________________________
> > > From: Matthias J. Sax <ma...@confluent.io>
> > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hey,
> > >
> > > there was quite a pause on this KIP discussion and in the mean time, a
> > > new design for incremental cooporative rebalance was suggested:
> > >
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> l+Cooperative+Rebalancing%3A+Support+and+Policies
> Incremental Cooperative Rebalancing: Support and Policies ...<
> https://cwiki.apache.org/confluence/display/KAFKA/Increm
> ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> cwiki.apache.org
> Rebalancing between distributed application processes in Apache Kafka was
> enhanced considerably when it was decoupled as logic from Kafka brokers and
> was moved as responsibility to the clients and specifically to Kafka
> Consumer. This pattern has been working robustly for quite a while now and
> has ...
>
>
>
> > Incremental Cooperative Rebalancing: Support and Policies ...<
> > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> l+Cooperative+Rebalancing%3A+Support+and+Policies
> > >
> > cwiki.apache.org
> > Rebalancing between distributed application processes in Apache Kafka was
> > enhanced considerably when it was decoupled as logic from Kafka brokers
> and
> > was moved as responsibility to the clients and specifically to Kafka
> > Consumer. This pattern has been working robustly for quite a while now
> and
> > has ...
> >
> >
> >
> > >
> > >
> > > We should make sure that the proposal and this KIP align to each other.
> > > Thoughts?
> > >
> > >
> > > -Matthias
> > >
> > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > Hey Mike,
> > > >
> > > >
> > > > thanks for the feedback, the two question are very thoughtful!
> > > >
> > > >
> > > >> 1) I am a little confused about the distinction for the leader. If
> the
> > > consumer node that was assigned leader does a bounce (goes down and
> > quickly
> > > comes up) to update application code, will a rebalance be triggered? I
> >
> > do
> > > not think a bounce of the leader should trigger a rebalance.
> > > >
> > > > For Q1 my intention was to minimize the change within one KIP, since
> > the
> > > leader rejoining case could be addressed separately.
> > > >
> > > >
> > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > gracefully increase the number of nodes in the cluster. I think we need
> > to
> > > support graceful shrink down as well. If I set the registration timeout
> > to
> > > 5 minutes > to handle rolling restarts or intermittent failures without
> > > shuffling state, I don't want to wait 5 minutes in order for the group
> to
> > > rebalance if I am intentionally removing a node from the cluster. I am
> > not
> > > sure the best way to > do this. One idea I had was adding the ability
> > for a
> > > CLI or Admin API to force a rebalance of the group. This would allow
> for
> > an
> > > admin to trigger the rebalance manually without waiting the entire
> > > registration timeout on > shrink down. What do you think?
> > > >
> > > > For 2) my understanding is that for scaling down case it is better to
> > be
> > > addressed by CLI tool than code logic, since only by human evaluation
> we
> > > could decide whether it is a "right timing" -- the time when all the
> > > scaling down consumers are offline -- to kick in rebalance. Unless we
> > > introduce another term on coordinator which indicates the target
> consumer
> > > group size, broker will find it hard to decide when to start rebalance.
> > So
> > > far I prefer to hold the implementation for that, but agree we could
> > > discuss whether we want to introduce admin API in this KIP or a
> separate
> > > one.
> > > >
> > > >
> > > > Thanks again for the proposed ideas!
> > > >
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Mike Freyberger <mi...@xandr.com>
> > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > specifying member id
> > > >
> > > > Boyang,
> > > >
> > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > >
> > > > 1) I am a little confused about the distinction for the leader. If
> the
> > > consumer node that was assigned leader does a bounce (goes down and
> > quickly
> > > comes up) to update application code, will a rebalance be triggered? I
> do
> > > not think a bounce of the leader should trigger a rebalance.
> > > >
> > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > gracefully increase the number of nodes in the cluster. I think we need
> > to
> > > support graceful shrink down as well. If I set the registration timeout
> > to
> > > 5 minutes to handle rolling restarts or intermittent failures without
> > > shuffling state, I don't want to wait 5 minutes in order for the group
> to
> > > rebalance if I am intentionally removing a node from the cluster. I am
> > not
> > > sure the best way to do this. One idea I had was adding the ability
> for a
> > > CLI or Admin API to force a rebalance of the group. This would allow
> for
> > an
> > > admin to trigger the rebalance manually without waiting the entire
> > > registration timeout on shrink down. What do you think?
> > > >
> > > > Mike
> > > >
> > > > ?On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > > >
> > > >     Btw, I updated KIP 345 based on my understanding. Feel free to
> take
> > > another round of look:
> > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > KIP-345: Introduce static membership protocol to reduce ...<
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > >
> > > > cwiki.apache.org
> > > > For stateful applications, one of the biggest performance bottleneck
> is
> > > the state shuffling. In Kafka consumer, there is a concept called
> > > "rebalance" which means that for given M partitions and N consumers in
> > one
> > > consumer group, Kafka will try to balance the load between consumers
> and
> > > ideally have ...
> > > >
> > > >
> > > >
> > > >
> > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > >
> > > >     cwiki.apache.org
> > > >     For stateful applications, one of the biggest performance
> > bottleneck
> > > is the state shuffling. In Kafka consumer, there is a concept called
> > > "rebalance" which means that for given M partitions and N consumers in
> > one
> > > consumer group, Kafka will try to balance the load between consumers
> and
> > > ideally have ...
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >     ________________________________
> > > >     From: Boyang Chen <bc...@outlook.com>
> > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > >     To: dev@kafka.apache.org
> > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > > by specifying member id
> > > >
> > > >     Thanks everyone for the input on this thread! (Sorry it's been a
> > > while) I feel that we are very close to the final solution.
> > > >
> > > >
> > > >     Hey Jason and Mike, I have two quick questions on the new
> features
> > > here:
> > > >
> > > >       1.  so our proposal is that until we add a new static member
> into
> > > the group (scale up), we will not trigger rebalance until the
> > "registration
> > > timeout"( the member has been offline for too long)? How about leader's
> > > rejoin request, I think we should still trigger rebalance when that
> > > happens, since the consumer group may have new topics to consume?
> > > >       2.  I'm not very clear on the scale up scenario in static
> > > membership here. Should we fallback to dynamic membership while
> > > adding/removing hosts (by setting member.name = null), or we still
> want
> > > to add instances with `member.name` so that we eventually
> expand/shrink
> > > the static membership? I personally feel the easier solution is to spin
> > up
> > > new members and wait until either the same "registration timeout" or a
> > > "scale up timeout" before starting the rebalance. What do you think?
> > > >
> > > >     Meanwhile I will go ahead to make changes to the KIP with our
> newly
> > > discussed items and details. Really excited to see the design has
> become
> > > more solid.
> > > >
> > > >     Best,
> > > >     Boyang
> > > >
> > > >     ________________________________
> > > >     From: Jason Gustafson <ja...@confluent.io>
> > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > >     To: dev
> > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > > by specifying member id
> > > >
> > > >     Hey Mike,
> > > >
> > > >     Yeah, that's a good point. A long "registration timeout" may not
> be
> > > a great
> > > >     idea. Perhaps in practice you'd set it long enough to be able to
> > > detect a
> > > >     failure and provision a new instance. Maybe on the order of 10
> > > minutes is
> > > >     more reasonable.
> > > >
> > > >     In any case, it's probably a good idea to have an administrative
> > way
> > > to
> > > >     force deregistration. One option is to extend the DeleteGroups
> API
> > > with a
> > > >     list of members names.
> > > >
> > > >     -Jason
> > > >
> > > >
> > > >
> > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > mfreyberger@appnexus.com>
> > > >     wrote:
> > > >
> > > >     > Jason,
> > > >     >
> > > >     > Regarding step 4 in your proposal which suggests beginning a
> long
> > > timer
> > > >     > (30 minutes) when a static member leaves the group, would there
> > > also be the
> > > >     > ability for an admin to force a static membership expiration?
> > > >     >
> > > >     > I'm thinking that during particular types of outages or
> upgrades
> > > users
> > > >     > would want forcefully remove a static member from the group.
> > > >     >
> > > >     > So the user would shut the consumer down normally, which
> wouldn't
> > > trigger
> > > >     > a rebalance. Then the user could use an admin CLI tool to force
> > > remove that
> > > >     > consumer from the group, so the TopicPartitions that were
> > > previously owned
> > > >     > by that consumer can be released.
> > > >     >
> > > >     > At a high level, we need consumer groups to gracefully handle
> > > intermittent
> > > >     > failures and permanent failures. Currently, the consumer group
> > > protocol
> > > >     > handles permanent failures well, but does not handle
> intermittent
> > > failures
> > > >     > well (it creates unnecessary rebalances). I want to make sure
> the
> > > overall
> > > >     > solution here handles both intermittent failures and permanent
> > > failures,
> > > >     > rather than sacrificing support for permanent failures in order
> > to
> > > provide
> > > >     > support for intermittent failures.
> > > >     >
> > > >     > Mike
> > > >     >
> > > >     > Sent from my iPhone
> > > >     >
> > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > jason@confluent.io>
> > > wrote:
> > > >     > >
> > > >     > > Hey Guozhang,
> > > >     > >
> > > >     > > Responses below:
> > > >     > >
> > > >     > > Originally I was trying to kill more birds with one stone
> with
> > > KIP-345,
> > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > shutting
> > > down a
> > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > email), and
> > > >     > >> hence proposing to have a pure static-membership protocol.
> But
> > > thinking
> > > >     > >> twice about it I now feel it may be too ambitious and worth
> > > fixing in
> > > >     > >> another KIP.
> > > >     > >
> > > >     > >
> > > >     > > I was considering an extension to support pre-initialization
> of
> > > the
> > > >     > static
> > > >     > > members of the group, but I agree we should probably leave
> this
> > > problem
> > > >     > for
> > > >     > > future work.
> > > >     > >
> > > >     > > 1. How this longish static member expiration timeout defined?
> > Is
> > > it via a
> > > >     > >> broker, hence global config, or via a client config which
> can
> > be
> > > >     > >> communicated to broker via JoinGroupRequest?
> > > >     > >
> > > >     > >
> > > >     > > I am not too sure. I tend to lean toward server-side configs
> > > because they
> > > >     > > are easier to evolve. If we have to add something to the
> > > protocol, then
> > > >     > > we'll be stuck with it forever.
> > > >     > >
> > > >     > > 2. Assuming that for static members, LEAVE_GROUP request will
> > not
> > > >     > trigger a
> > > >     > >> rebalance immediately either, similar to session timeout,
> but
> > > only the
> > > >     > >> longer member expiration timeout, can we remove the
> internal "
> > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > walk-around
> > > >     > then?
> > > >     > >
> > > >     > >
> > > >     > > Yeah, I hope we can ultimately get rid of it, but we may need
> > it
> > > for
> > > >     > > compatibility with older brokers. A related question is what
> > > should be
> > > >     > the
> > > >     > > behavior of the consumer if `member.name` is provided but
> the
> > > broker
> > > >     > does
> > > >     > > not support it? We could either fail or silently downgrade to
> > > dynamic
> > > >     > > membership.
> > > >     > >
> > > >     > > -Jason
> > > >     > >
> > > >     > >
> > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > >     > wrote:
> > > >     > >>
> > > >     > >> Hey Jason,
> > > >     > >>
> > > >     > >> I like your idea to simplify the upgrade protocol to allow
> > > co-exist of
> > > >     > >> static and dynamic members. Admittedly it may make the
> > > coordinator-side
> > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > >     > >>
> > > >     > >> Originally I was trying to kill more birds with one stone
> with
> > > KIP-345,
> > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > shutting
> > > down a
> > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > email), and
> > > >     > >> hence proposing to have a pure static-membership protocol.
> But
> > > thinking
> > > >     > >> twice about it I now feel it may be too ambitious and worth
> > > fixing in
> > > >     > >> another KIP. With that, I think what you've proposed here
> is a
> > > good way
> > > >     > to
> > > >     > >> go for KIP-345 itself.
> > > >     > >>
> > > >     > >> Note there are a few details in your proposal we'd still
> need
> > > to figure
> > > >     > >> out:
> > > >     > >>
> > > >     > >> 1. How this longish static member expiration timeout
> defined?
> > > Is it via
> > > >     > a
> > > >     > >> broker, hence global config, or via a client config which
> can
> > be
> > > >     > >> communicated to broker via JoinGroupRequest?
> > > >     > >>
> > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> will
> > > not
> > > >     > trigger a
> > > >     > >> rebalance immediately either, similar to session timeout,
> but
> > > only the
> > > >     > >> longer member expiration timeout, can we remove the
> internal "
> > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > walk-around
> > > >     > then?
> > > >     > >>
> > > >     > >>
> > > >     > >>
> > > >     > >> Guozhang
> > > >     > >>
> > > >     > >>
> > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > jason@confluent.io>
> > > >     > >> wrote:
> > > >     > >>
> > > >     > >>> Hey All,
> > > >     > >>>
> > > >     > >>> Nice to see some solid progress on this. It sounds like one
> > of
> > > the
> > > >     > >>> complications is allowing static and dynamic registration
> to
> > > coexist.
> > > >     > I'm
> > > >     > >>> wondering if we can do something like the following:
> > > >     > >>>
> > > >     > >>> 1. Statically registered members (those joining the group
> > with
> > > a
> > > >     > >> non-null `
> > > >     > >>> member.name`) maintain a session with the coordinator just
> > > like
> > > >     > dynamic
> > > >     > >>> members.
> > > >     > >>> 2. If a session is active for a static member when a
> > rebalance
> > > begins,
> > > >     > >> then
> > > >     > >>> basically we'll keep the current behavior. The rebalance
> will
> > > await the
> > > >     > >>> static member joining the group.
> > > >     > >>> 3. If a static member does not have an active session, then
> > the
> > > >     > >> coordinator
> > > >     > >>> will not wait for it to join, but will still include it in
> > the
> > > >     > rebalance.
> > > >     > >>> The coordinator will forward the cached subscription
> > > information to the
> > > >     > >>> leader and will cache the assignment after the rebalance
> > > completes.
> > > >     > (Note
> > > >     > >>> that we still have the generationId to fence offset commits
> > > from a
> > > >     > static
> > > >     > >>> zombie if the assignment changes.)
> > > >     > >>> 4. When a static member leaves the group or has its session
> > > expire, no
> > > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > > expire the
> > > >     > >> static
> > > >     > >>> registration. This would be a longish timeout (like 30
> > minutes
> > > say).
> > > >     > >>>
> > > >     > >>> So basically static members participate in all rebalances
> > > regardless
> > > >     > >>> whether they have an active session. In a given rebalance,
> > > some of the
> > > >     > >>> members may be static and some dynamic. The group leader
> can
> > > >     > >> differentiate
> > > >     > >>> the two based on the presence of the `member.name` (we
> have
> > > to add
> > > >     > this
> > > >     > >> to
> > > >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> > > leaders
> > > >     > >>> preferentially from the active members that support the
> > latest
> > > >     > JoinGroup
> > > >     > >>> protocol and are using static membership. If we have to
> > choose
> > > a leader
> > > >     > >>> with an old version, however, it would see all members in
> the
> > > group
> > > >     > >> (static
> > > >     > >>> or dynamic) as dynamic members and perform the assignment
> as
> > > usual.
> > > >     > >>>
> > > >     > >>> Would that work?
> > > >     > >>>
> > > >     > >>> -Jason
> > > >     > >>>
> > > >     > >>>
> > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > >     > >> wrote:
> > > >     > >>>
> > > >     > >>>> Hello Boyang,
> > > >     > >>>>
> > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > >     > >>>>
> > > >     > >>>> 1. Where will "change-group-timeout" be communicated to
> the
> > > broker?
> > > >     > >> Will
> > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> going
> > > to
> > > >     > >>> piggy-back
> > > >     > >>>> on the existing session-timeout field (assuming that the
> > > original
> > > >     > value
> > > >     > >>>> will not be used anywhere in the static membership any
> > more)?
> > > >     > >>>>
> > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > timeout to
> > > >     > >> return,
> > > >     > >>>> we shall still trigger rebalance but it could still try to
> > > catch
> > > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> > your
> > > proposal
> > > >     > >> is
> > > >     > >>>> that for static memberships, the broker will NOT trigger
> > > rebalance
> > > >     > even
> > > >     > >>>> after session-timeout has been detected, but only that
> after
> > > >     > >>>> change-group-timeout
> > > >     > >>>> which is supposed to be longer than session-timeout to be
> > > defined?
> > > >     > >>>>
> > > >     > >>>> 3. "A join group request with member.name set will be
> > > treated as
> > > >     > >>>> `static-membership` strategy", in this case, how would the
> > > switch from
> > > >     > >>>> dynamic to static happen, since whoever changed the
> > > member.name to
> > > >     > >>>> not-null
> > > >     > >>>> will be rejected, right?
> > > >     > >>>>
> > > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > > timeout to
> > > >     > >>> trigger
> > > >     > >>>> rebalance should be sufficient." this is also a bit
> unclear
> > > to me: who
> > > >     > >>> will
> > > >     > >>>> erase the cached mapping? Since it is on the broker-side I
> > > assume that
> > > >     > >>>> broker has to do it. Are you suggesting to use a new
> request
> > > for it?
> > > >     > >>>>
> > > >     > >>>> 5. "Halfway switch": following 3) above, if your proposal
> is
> > > basically
> > > >     > >> to
> > > >     > >>>> let "first join-request wins", and the strategy will stay
> as
> > > is until
> > > >     > >> all
> > > >     > >>>> members are gone, then this will also not happen since
> > > whoever used
> > > >     > >>>> different strategy as the first guy who sends join-group
> > > request will
> > > >     > >> be
> > > >     > >>>> rejected right?
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> Guozhang
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > john@confluent.io>
> > > >     > >> wrote:
> > > >     > >>>>
> > > >     > >>>>> This sounds good to me!
> > > >     > >>>>>
> > > >     > >>>>> Thanks for the time you've spent on it,
> > > >     > >>>>> -John
> > > >     > >>>>>
> > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > bchen11@outlook.com>
> > > >     > >>>> wrote:
> > > >     > >>>>>
> > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently
> > and
> > > >     > >> haven't
> > > >     > >>>> got
> > > >     > >>>>>> time to update this thread. To summarize what we come up
> > so
> > > far,
> > > >     > >> here
> > > >     > >>>> is
> > > >     > >>>>> a
> > > >     > >>>>>> draft updated plan:
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Introduce a new config called `member.name` which is
> > > supposed to
> > > >     > >> be
> > > >     > >>>>>> provided uniquely by the consumer client. The broker
> will
> > > maintain
> > > >     > >> a
> > > >     > >>>>> cache
> > > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > > request with
> > > >     > >>>>>> member.name set will be treated as `static-membership`
> > > strategy,
> > > >     > >> and
> > > >     > >>>>> will
> > > >     > >>>>>> reject any join group request without member.name. So
> > this
> > > >     > >>>> coordination
> > > >     > >>>>>> change will be differentiated from the
> > `dynamic-membership`
> > > >     > >> protocol
> > > >     > >>> we
> > > >     > >>>>>> currently have.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When handling static join group request:
> > > >     > >>>>>>
> > > >     > >>>>>>  1.   The broker will check the membership to see
> whether
> > > this is
> > > >     > >> a
> > > >     > >>>> new
> > > >     > >>>>>> member. If new, broker allocate a unique member id,
> cache
> > > the
> > > >     > >> mapping
> > > >     > >>>> and
> > > >     > >>>>>> move to rebalance stage.
> > > >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> > > will not
> > > >     > >>>> change
> > > >     > >>>>>> group state, and return its cached member.id and
> current
> > > >     > >> assignment.
> > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin
> with
> > > pair
> > > >     > >>> member
> > > >     > >>>>>> name and id, I think for join group request it is ok to
> > > leave
> > > >     > >> member
> > > >     > >>> id
> > > >     > >>>>>> blank as member name is the unique identifier. In commit
> > > offset
> > > >     > >>> request
> > > >     > >>>>> we
> > > >     > >>>>>> *must* have both.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When handling commit offset request, if enabled with
> > static
> > > >     > >>> membership,
> > > >     > >>>>>> each time the commit request must have both member.name
> > and
> > > >     > >>> member.id
> > > >     > >>>> to
> > > >     > >>>>>> be identified as a `certificated member`. If not, this
> > > means there
> > > >     > >>> are
> > > >     > >>>>>> duplicate consumer members with same member name and the
> > > request
> > > >     > >> will
> > > >     > >>>> be
> > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> client
> > > will
> > > >     > >> send a
> > > >     > >>>>>> leave group request (static membership mode). In static
> > > membership,
> > > >     > >>> we
> > > >     > >>>>> will
> > > >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> > > provided by
> > > >     > >>>>> leader.
> > > >     > >>>>>> So we will wait for all the members to rejoin the group
> > and
> > > do
> > > >     > >>> exactly
> > > >     > >>>>> one
> > > >     > >>>>>> rebalance since all members are expected to rejoin
> within
> > > timeout.
> > > >     > >> If
> > > >     > >>>>>> consumer crashes, the join group request from the
> > restarted
> > > >     > >> consumer
> > > >     > >>>> will
> > > >     > >>>>>> be recognized as an existing member and be handled as
> > above
> > > >     > >> condition
> > > >     > >>>> 1;
> > > >     > >>>>>> However, if the consumer takes longer than session
> timeout
> > > to
> > > >     > >> return,
> > > >     > >>>> we
> > > >     > >>>>>> shall still trigger rebalance but it could still try to
> > > catch
> > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > timeout, its
> > > >     > >>>> cached
> > > >     > >>>>>> state on broker will be garbage collected and trigger a
> > new
> > > >     > >> rebalance
> > > >     > >>>>> when
> > > >     > >>>>>> it finally joins.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> And consider the switch between dynamic to static
> > > membership.
> > > >     > >>>>>>
> > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise
> the
> > > >     > >> membership
> > > >     > >>>> to
> > > >     > >>>>>> static and wait for all the current members to restart,
> > > since their
> > > >     > >>>>>> membership is still dynamic. Here our assumption is that
> > the
> > > >     > >> restart
> > > >     > >>>>>> process shouldn't take a long time, as long restart is
> > > breaking the
> > > >     > >>>>>> `rebalance timeout` in whatever membership protocol we
> are
> > > using.
> > > >     > >>>> Before
> > > >     > >>>>>> restart, all dynamic member join requests will be
> > rejected.
> > > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> > which
> > > should
> > > >     > >>> be
> > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > session
> > > timeout
> > > >     > >>> to
> > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > current
> > > >     > >>> behavior)
> > > >     > >>>>>>  3.  Halfway switch: a corner case is like some clients
> > keep
> > > >     > >> dynamic
> > > >     > >>>>>> membership while some keep static membership. This will
> > > cause the
> > > >     > >>> group
> > > >     > >>>>>> rebalance forever without progress because
> dynamic/static
> > > states
> > > >     > >> are
> > > >     > >>>>>> bouncing each other. This could guarantee that we will
> not
> > > make the
> > > >     > >>>>>> consumer group work in a wrong state by having half
> static
> > > and half
> > > >     > >>>>> dynamic.
> > > >     > >>>>>>
> > > >     > >>>>>> To guarantee correctness, we will also push the member
> > > name/id pair
> > > >     > >>> to
> > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > > upgrade the
> > > >     > >> API
> > > >     > >>>>>> version, these details will be further discussed back in
> > > the KIP.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Are there any concern for this high level proposal? Just
> > > want to
> > > >     > >>>>> reiterate
> > > >     > >>>>>> on the core idea of the KIP: "If the broker recognize
> this
> > > consumer
> > > >     > >>> as
> > > >     > >>>> an
> > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > >     > >>>>>>
> > > >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal
> is
> > > much
> > > >     > >> more
> > > >     > >>>>>> robust than previous one!
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Best,
> > > >     > >>>>>>
> > > >     > >>>>>> Boyang
> > > >     > >>>>>>
> > > >     > >>>>>> ________________________________
> > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > >     > >>>>>> To: dev@kafka.apache.org
> > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > >     > >>> by
> > > >     > >>>>>> specifying member id
> > > >     > >>>>>>
> > > >     > >>>>>> Hi,
> > > >     > >>>>>>
> > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> about
> > > internals
> > > >     > >>>> again
> > > >     > >>>>>> :)
> > > >     > >>>>>>
> > > >     > >>>>>> I like the idea or a user config `member.name` and to
> > keep
> > > `
> > > >     > >>> member.id`
> > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > client.id`
> > > might
> > > >     > >>> not
> > > >     > >>>>>> be a good idea.
> > > >     > >>>>>>
> > > >     > >>>>>> To clarify the algorithm, each time we generate a new `
> > > member.id`,
> > > >     > >>> we
> > > >     > >>>>>> also need to update the "group membership" information
> > (ie,
> > > mapping
> > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id
> `
> > > replaces
> > > >     > >>> the
> > > >     > >>>>>> old entry in the cache.
> > > >     > >>>>>>
> > > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > > member.id`
> > > >     > >>>> mapping
> > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> > > this IMHO.
> > > >     > >>>>>>
> > > >     > >>>>>> For changing the default value of config
> > > `leave.group.on.close`. I
> > > >     > >>>> agree
> > > >     > >>>>>> with John, that we should not change the default config,
> > > because it
> > > >     > >>>>>> would impact all consumer groups with dynamic
> assignment.
> > > However,
> > > >     > >> I
> > > >     > >>>>>> think we can document, that if static assignment is used
> > > (ie,
> > > >     > >>>>>> `member.name` is configured) we never send a
> > > LeaveGroupRequest
> > > >     > >>>>>> regardless of the config. Note, that the config is
> > > internal, so not
> > > >     > >>>> sure
> > > >     > >>>>>> how to document this in detail. We should not expose the
> > > internal
> > > >     > >>>> config
> > > >     > >>>>>> in the docs.
> > > >     > >>>>>>
> > > >     > >>>>>> About upgrading: why do we need have two rolling bounces
> > > and encode
> > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > >     > >>>>>>
> > > >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> > > static, I
> > > >     > >>>> don't
> > > >     > >>>>>> see any reason why both should not work together and
> > single
> > > rolling
> > > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > > consumer and
> > > >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> > > and the
> > > >     > >>> broker
> > > >     > >>>>>> registers the [member.name, member.id] in the cache.
> Why
> > > would
> > > >     > >> this
> > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > assignment?
> > > >     > >>>>>>
> > > >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> > > need to
> > > >     > >> check
> > > >     > >>>> if
> > > >     > >>>>>> the mapping [member.name, member.id] contains the send
> `
> > > member.id`
> > > >     > >>> --
> > > >     > >>>> I
> > > >     > >>>>>> don't think this is necessary -- it seems to be
> sufficient
> > > to check
> > > >     > >>> the
> > > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as
> > be
> > > do
> > > >     > >> today
> > > >     > >>> --
> > > >     > >>>>>> thus, checking `member.id` does not require any change
> > > IMHO.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> -Matthias
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > >     > >>>>>>> @James
> > > >     > >>>>>>>
> > > >     > >>>>>>> What you described is true: the transition from dynamic
> > to
> > > static
> > > >     > >>>>>>> memberships are not thought through yet. But I do not
> > > think it is
> > > >     > >>> an
> > > >     > >>>>>>> impossible problem: note that we indeed moved the
> offset
> > > commit
> > > >     > >>> from
> > > >     > >>>> ZK
> > > >     > >>>>>> to
> > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> > > first to
> > > >     > >>>>>>> double-commits on both zk and coordinator, and then do
> a
> > > second
> > > >     > >>> round
> > > >     > >>>>> to
> > > >     > >>>>>>> turn the zk off.
> > > >     > >>>>>>>
> > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > >     > >>>> two-rolling-bounce
> > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> > > "static"
> > > >     > >>> while
> > > >     > >>>>>> keep
> > > >     > >>>>>>> the registry-id field empty still, in this case, the
> > > coordinator
> > > >     > >>>> still
> > > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> > while
> > > >     > >>> allowing
> > > >     > >>>>> the
> > > >     > >>>>>>> protocol to be set to "static"; after the first rolling
> > > bounce,
> > > >     > >> the
> > > >     > >>>>> group
> > > >     > >>>>>>> protocol is already "static", then a second rolling
> > bounce
> > > is
> > > >     > >>>> triggered
> > > >     > >>>>>> and
> > > >     > >>>>>>> this time we set the registry-id.
> > > >     > >>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>> Guozhang
> > > >     > >>>>>>>
> > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > >     > >> wushujames@gmail.com>
> > > >     > >>>>>> wrote:
> > > >     > >>>>>>>
> > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> this:
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > >     > >>>
> > > >     > >>>>> wrote:
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > > fields:
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> membership
> > > >     > >>> protocols.
> > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > pre-defined
> > > >     > >>> member
> > > >     > >>>>> id.
> > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> optional
> > > >     > >>>>>>>>> "group-change-timeout" value.
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> > > protocols
> > > >     > >>> for
> > > >     > >>>>> all
> > > >     > >>>>>>>>> group members: we accept the protocol on the first
> > joined
> > > >     > >> member
> > > >     > >>> of
> > > >     > >>>>> the
> > > >     > >>>>>>>>> group, and if later joining members indicate a
> > different
> > > >     > >>> membership
> > > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > > value was
> > > >     > >>>>> different
> > > >     > >>>>>>>> to
> > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > application that
> > > >     > >>>> wants
> > > >     > >>>>>> to
> > > >     > >>>>>>>> switch to using static membership? Let's say there are
> > 10
> > > >     > >>> instances
> > > >     > >>>> of
> > > >     > >>>>>> it.
> > > >     > >>>>>>>> As the instances go through a rolling restart, they
> will
> > > switch
> > > >     > >>> from
> > > >     > >>>>>>>> dynamic membership (the default?) to static
> membership.
> > > As each
> > > >     > >>> one
> > > >     > >>>>>> leaves
> > > >     > >>>>>>>> the group and restarts, they will be rejected from the
> > > group
> > > >     > >>>> (because
> > > >     > >>>>>> the
> > > >     > >>>>>>>> group is currently using dynamic membership). The
> group
> > > will
> > > >     > >>> shrink
> > > >     > >>>>> down
> > > >     > >>>>>>>> until there is 1 node handling all the traffic. After
> > > that one
> > > >     > >>>>> restarts,
> > > >     > >>>>>>>> the group will switch over to static membership.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> Is that right? That means that the transition plan
> from
> > > dynamic
> > > >     > >> to
> > > >     > >>>>>> static
> > > >     > >>>>>>>> membership isn't very smooth.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> I'm not really sure what can be done in this case.
> This
> > > reminds
> > > >     > >> me
> > > >     > >>>> of
> > > >     > >>>>>> the
> > > >     > >>>>>>>> transition plans that were discussed for moving from
> > > >     > >>> zookeeper-based
> > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That
> was
> > > also
> > > >     > >>> hard,
> > > >     > >>>>> and
> > > >     > >>>>>>>> ultimately we decided not to build that.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> -James
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> --
> > > >     > >>>> -- Guozhang
> > > >     > >>>>
> > > >     > >>>
> > > >     > >>
> > > >     > >>
> > > >     > >>
> > > >     > >> --
> > > >     > >> -- Guozhang
> > > >     > >>
> > > >     >
> > > >
> > > >
> > >
> > >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Jason Gustafson <ja...@confluent.io>.
Hey Boyang,

Thanks for the updates. From a high level, I think this actually
complements Konstantine's writeup on incremental rebalancing. The gap we're
addressing is providing a way to bind the the partition assignment of a
group to a set of user-provided ids so that we are not so reliant on the
group's immediate state. For example, these ids might identify the state
store volume for particular streams instances. This is basically what you
need to work well with k8s stateful sets (as far as I understand them).

One key decision is how we would define and update the expected static
members in a consumer group. The mechanics of the registration and
expansion timeouts feel a little bit clunky. For the sake of discussion, I
was wondering if we could just say that static members do not expire.
Instead, we offer an admin API that lets a user define the expected members
of the group. This API could be used to both grow and shrink a group. This
would solve the rebalancing problems when applications are initially
bootstrapped or when they are restarted because we would always know how
many members should be in a group. What do you think?

By the way, it would be helpful to include the full schema definition for
any protocol changes in the proposal.

Thanks,
Jason


On Mon, Nov 12, 2018 at 8:56 AM, Boyang Chen <bc...@outlook.com> wrote:

> Thanks Mayuresh for the feedback! Do you have a quick example for passing
> in consumer config dynamically? I mainly use Kafka Streams at my daily work
> so probably missing the idea how to do it in the current consumer setting.
>
>
> For differentiating session timeout and registration timeout, I would try
> to enhance the documentation in the first stage to see how people react to
> the confusion (would be great if they feel straightforward!). Since one
> doesn't have to fully understand the difference unless defining the new
> config "member name", for current users we could buy some time to listen to
> their understandings and improve our documentation correspondingly in the
> follow-up KIPs.
>
>
> Boyang
>
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Sunday, November 11, 2018 1:06 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for the reply.
>
> Please find the replies inline below :
> For having a consumer config at runtime, I think it's not necessary to
> address in this KIP because most companies run sidecar jobs through daemon
> software like puppet. It should be easy to change the config through script
> or UI without actual code change. We still want to leave flexibility for
> user to define member name as they like.
> ---- This might be little different for companies that use configuration
> management tools that does not allow the applications to define/change the
> configs dynamically. For example, if we use something similar to spring to
> pull in the configs for the KafkaConsumer and pass it to the constructor to
> create the KafkaConsumer object, it will be hard to specify a unique value
> to the "MEMBER_NAME" config unless someone deploying the app generates a
> unique string for this config outside the deployment workflow and copies it
> statically before starting up each consumer instance. Unless we can loosen
> the criteria for uniqueness of this config value, for each consumer
> instance in the consumer group, I am not sure of a better way of
> addressing this. If we don't want to loosen the criteria, then providing a
> dynamic way to pass this in at runtime, would put the onus of having the
> same unique value each time a consumer is restarted, on to the application
> that is running the consumer.
>
> I just updated the kip about having both "registration timeout" and
> "session timeout". The benefit of having two configs instead of one is to
> reduce the mental burden for operation, for example user just needs to
> unset "member name" to cast back to dynamic membership without worrying
> about tuning the "session timeout" back to a smaller value.
> --- That is a good point. I was thinking, if both the configs are
> specified, it would be confusing for the end user without understanding the
> internals of the consumer and its interaction with group coordinator, as
> which takes precedence when and how it affects the consumer behavior. Just
> my 2 cents.
>
> Thanks,
>
> Mayuresh
>
> On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:
>
> > Hey Mayuresh,
> >
> >
> > thanks for the thoughtful questions! Let me try to answer your questions
> > one by one.
> >
> >
> > For having a consumer config at runtime, I think it's not necessary to
> > address in this KIP because most companies run sidecar jobs through
> daemon
> > software like puppet. It should be easy to change the config through
> script
> > or UI without actual code change. We still want to leave flexibility for
> > user to define member name as they like.
> >
> >
> > I just updated the kip about having both "registration timeout" and
> > "session timeout". The benefit of having two configs instead of one is to
> > reduce the mental burden for operation, for example user just needs to
> > unset "member name" to cast back to dynamic membership without worrying
> > about tuning the "session timeout" back to a smaller value.
> >
> >
> > For backup topic, I think it's a low-level detail which could be
> addressed
> > in the implementation. I feel no preference of adding a new topic vs
> reuse
> > consumer offsets topic. I will do more analysis and make a trade-off
> > comparison. Nice catch!
> >
> >
> > I hope the explanations make sense to you. I will keep polishing on the
> > edge cases and details.
> >
> >
> > Best,
> >
> > Boyang
> >
> > ________________________________
> > From: Mayuresh Gharat <gh...@gmail.com>
> > Sent: Saturday, November 10, 2018 10:25 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hi Boyang,
> >
> > Thanks for the KIP and sorry for being late to the party. This KIP is
> > really useful for us at Linkedin.
> >
> > I had a few questions :
> >
> > The idea of having static member name seems nice, but instead of a
> config,
> > would it be possible for it to be passed in to the consumer at runtime?
> > This is because an app might want to decide the config value at runtime
> > using its host information for example, to generate the unique member
> name.
> >
> > Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> > wondering if we can reuse the session timeout here. This might help us to
> > have one less config on the consumer.
> >
> > The KIP also talks about adding another internal topic
> "static_member_map".
> > Would the semantics (GroupCoordinator broker, topic configs) be the same
> as
> > __consumer_offsets topic?
> >
> > Thanks,
> >
> > Mayuresh
> >
> >
> > On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:
> >
> > > I took a quick pass of the proposal. First I would say it's a very
> > > brilliant initiative from Konstantine and Confluent folks. To draft up
> a
> > > proposal like this needs deep understanding of the rebalance protocol!
> I
> > > summarized some thoughts here.
> > >
> > >
> > > Overall the motivations of the two proposals align on that:
> > >
> > >   1.  Both believe the invariant resource (belonging to the same
> process)
> > > should be preserved across rebalance.
> > >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > > redistribution. I don't use rebalance here since part one of the
> > > cooperative proposal could potentially introduce more rebalances but
> only
> > > on must-move resources.
> > >   3.  Scale up/down and rolling bounce are causing unnecessary resource
> > > shuffling that need to be mitigated.
> > >
> > >
> > > On motivation level, I think both approach could solve/mitigate the
> above
> > > issues. They are just different in design philosophy, or I would say
> the
> > > perspective difference between framework user and algorithm designer.
> > >
> > >
> > > Two proposals have different focuses. KIP-345 is trying to place more
> > > fine-grained control on the broker side to reduce the unnecessary
> > > rebalances, while keeping the client logic intact. This is pretty
> > intuitive
> > > cause-effect for normal developers who are not very familiar with
> > rebalance
> > > protocol. As a developer working with Kafka Streams daily, I'd be happy
> > to
> > > see a simplified rebalance protocol and just focus on maintaining the
> > > stream/consumer jobs. Too many rebalances raised my concern on the job
> > > health. To be concise, static membership has the advantage of reducing
> > > mental burden.
> > >
> > >
> > > Cooperative proposal takes thoughtful approach on client side. We want
> to
> > > have fine-grained control on the join/exit group behaviors and make the
> > > current dynamic membership better to address above issues. I do feel
> our
> > > idea crossed on the delayed rebalance when we scale up/down, which
> could
> > > potentially reduce the state shuffling and decouple the behavior from
> > > session timeout which is already overloaded.  In this sense, I believe
> > both
> > > approaches would serve well in making "reasonable rebalance" happen at
> > the
> > > "right timing".
> > >
> > >
> > > However, based on my understanding, either 345 or cooperative
> rebalancing
> > > is not solving the problem Mike has proposed: could we do a better job
> at
> > > scaling up/down in ideal timing? My initial response was to introduce
> an
> > > admin API which now I feel is sub-optimal, in that the goal of smooth
> > > transition is to make sure the newly up hosts are actually "ready". For
> > > example:
> > >
> > >
> > > We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> > > time we would like to scale up to 8 hosts, with the current
> improvements
> > we
> > > could reduce 4 potential rebalances to a single one. But the new hosts
> > are
> > > yet unknown to be "ready" if they need to reconstruct the local state.
> To
> > > be actually ready, we need 4 standby tasks running on those empty hosts
> > and
> > > leader needs to wait for the signal of "replay/reconstruct complete" to
> > > actually involve them into the main consumer group. Otherwise,
> rebalance
> > > just kills our performance since we need to wait indefinite long for
> task
> > > migration.
> > >
> > >
> > > The scale down is also tricky such that we are not able to define a
> > "true"
> > > leave of a member. Rebalance immediately after "true" leaves are most
> > > optimal comparing with human intervention. Does this make sense?
> > >
> > >
> > > My intuition is that cooperative approach which was implemented on the
> > > client side could better handle scaling cases than KIP 345, since it
> > > involves a lot of algorithmic changes to define "replaying" stage,
> which
> > I
> > > feel would over-complicate broker logic if implemented on coordinator.
> If
> > > we let 345 focus on reducing unnecessary rebalance, and let cooperative
> > > approach focus on judging best timing of scale up/down, the two efforts
> > > could be aligned. In long term, I feel the more complex improvement of
> > > consumer protocol should happen on client side instead of server side
> > which
> > > is easier to test and has less global impact for the entire Kafka
> > > production cluster.
> > >
> > >
> > > Thanks again to Konstantine, Matthias and other folks in coming up with
> > > this great client proposal. This is great complementation to KIP 345.
> In
> > a
> > > high level, we are not having any collision on the path and both
> > proposals
> > > are making sense here. Just need better sync to avoid duplicate effort
> :)
> > >
> > >
> > > Best,
> > >
> > > Boyang
> > >
> > >
> > > ________________________________
> > > From: Boyang Chen <bc...@outlook.com>
> > > Sent: Wednesday, November 7, 2018 1:57 PM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > > deeper look and make a comparison between the two proposals.
> > >
> > >
> > > Meanwhile for the scale down specifically for stateful streaming, we
> > could
> > > actually introduce a new status called "learner" where the newly up
> hosts
> > > could try to catch up with the assigned task progress first before
> > > triggering the rebalance, from which we don't see a sudden dip on the
> > > progress. However, it is built on top of the success of KIP-345.
> > >
> > >
> > > ________________________________
> > > From: Matthias J. Sax <ma...@confluent.io>
> > > Sent: Wednesday, November 7, 2018 7:02 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > > specifying member id
> > >
> > > Hey,
> > >
> > > there was quite a pause on this KIP discussion and in the mean time, a
> > > new design for incremental cooporative rebalance was suggested:
> > >
> > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> l+Cooperative+Rebalancing%3A+Support+and+Policies
> Incremental Cooperative Rebalancing: Support and Policies ...<
> https://cwiki.apache.org/confluence/display/KAFKA/Increm
> ental+Cooperative+Rebalancing%3A+Support+and+Policies>
> cwiki.apache.org
> Rebalancing between distributed application processes in Apache Kafka was
> enhanced considerably when it was decoupled as logic from Kafka brokers and
> was moved as responsibility to the clients and specifically to Kafka
> Consumer. This pattern has been working robustly for quite a while now and
> has ...
>
>
>
> > Incremental Cooperative Rebalancing: Support and Policies ...<
> > https://cwiki.apache.org/confluence/display/KAFKA/Incrementa
> l+Cooperative+Rebalancing%3A+Support+and+Policies
> > >
> > cwiki.apache.org
> > Rebalancing between distributed application processes in Apache Kafka was
> > enhanced considerably when it was decoupled as logic from Kafka brokers
> and
> > was moved as responsibility to the clients and specifically to Kafka
> > Consumer. This pattern has been working robustly for quite a while now
> and
> > has ...
> >
> >
> >
> > >
> > >
> > > We should make sure that the proposal and this KIP align to each other.
> > > Thoughts?
> > >
> > >
> > > -Matthias
> > >
> > > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > > Hey Mike,
> > > >
> > > >
> > > > thanks for the feedback, the two question are very thoughtful!
> > > >
> > > >
> > > >> 1) I am a little confused about the distinction for the leader. If
> the
> > > consumer node that was assigned leader does a bounce (goes down and
> > quickly
> > > comes up) to update application code, will a rebalance be triggered? I
> >
> > do
> > > not think a bounce of the leader should trigger a rebalance.
> > > >
> > > > For Q1 my intention was to minimize the change within one KIP, since
> > the
> > > leader rejoining case could be addressed separately.
> > > >
> > > >
> > > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > > gracefully increase the number of nodes in the cluster. I think we need
> > to
> > > support graceful shrink down as well. If I set the registration timeout
> > to
> > > 5 minutes > to handle rolling restarts or intermittent failures without
> > > shuffling state, I don't want to wait 5 minutes in order for the group
> to
> > > rebalance if I am intentionally removing a node from the cluster. I am
> > not
> > > sure the best way to > do this. One idea I had was adding the ability
> > for a
> > > CLI or Admin API to force a rebalance of the group. This would allow
> for
> > an
> > > admin to trigger the rebalance manually without waiting the entire
> > > registration timeout on > shrink down. What do you think?
> > > >
> > > > For 2) my understanding is that for scaling down case it is better to
> > be
> > > addressed by CLI tool than code logic, since only by human evaluation
> we
> > > could decide whether it is a "right timing" -- the time when all the
> > > scaling down consumers are offline -- to kick in rebalance. Unless we
> > > introduce another term on coordinator which indicates the target
> consumer
> > > group size, broker will find it hard to decide when to start rebalance.
> > So
> > > far I prefer to hold the implementation for that, but agree we could
> > > discuss whether we want to introduce admin API in this KIP or a
> separate
> > > one.
> > > >
> > > >
> > > > Thanks again for the proposed ideas!
> > > >
> > > >
> > > > Boyang
> > > >
> > > > ________________________________
> > > > From: Mike Freyberger <mi...@xandr.com>
> > > > Sent: Monday, November 5, 2018 6:13 AM
> > > > To: dev@kafka.apache.org
> > > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by
> > > specifying member id
> > > >
> > > > Boyang,
> > > >
> > > > Thanks for updating the KIP. It's shaping up well. Two things:
> > > >
> > > > 1) I am a little confused about the distinction for the leader. If
> the
> > > consumer node that was assigned leader does a bounce (goes down and
> > quickly
> > > comes up) to update application code, will a rebalance be triggered? I
> do
> > > not think a bounce of the leader should trigger a rebalance.
> > > >
> > > > 2) The timeout for shrink up makes a lot of sense and allows to
> > > gracefully increase the number of nodes in the cluster. I think we need
> > to
> > > support graceful shrink down as well. If I set the registration timeout
> > to
> > > 5 minutes to handle rolling restarts or intermittent failures without
> > > shuffling state, I don't want to wait 5 minutes in order for the group
> to
> > > rebalance if I am intentionally removing a node from the cluster. I am
> > not
> > > sure the best way to do this. One idea I had was adding the ability
> for a
> > > CLI or Admin API to force a rebalance of the group. This would allow
> for
> > an
> > > admin to trigger the rebalance manually without waiting the entire
> > > registration timeout on shrink down. What do you think?
> > > >
> > > > Mike
> > > >
> > > > On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > > >
> > > >     Btw, I updated KIP 345 based on my understanding. Feel free to
> take
> > > another round of look:
> > > >
> > > >
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > > KIP-345: Introduce static membership protocol to reduce ...<
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > >
> > > > cwiki.apache.org
> > > > For stateful applications, one of the biggest performance bottleneck
> is
> > > the state shuffling. In Kafka consumer, there is a concept called
> > > "rebalance" which means that for given M partitions and N consumers in
> > one
> > > consumer group, Kafka will try to balance the load between consumers
> and
> > > ideally have ...
> > > >
> > > >
> > > >
> > > >
> > > >     KIP-345: Introduce static membership protocol to reduce ...<
> > >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A
> +Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > >
> > > >     cwiki.apache.org
> > > >     For stateful applications, one of the biggest performance
> > bottleneck
> > > is the state shuffling. In Kafka consumer, there is a concept called
> > > "rebalance" which means that for given M partitions and N consumers in
> > one
> > > consumer group, Kafka will try to balance the load between consumers
> and
> > > ideally have ...
> > > >
> > > >
> > > >
> > > >
> > > >
> > > >     ________________________________
> > > >     From: Boyang Chen <bc...@outlook.com>
> > > >     Sent: Monday, October 29, 2018 12:34 PM
> > > >     To: dev@kafka.apache.org
> > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > > by specifying member id
> > > >
> > > >     Thanks everyone for the input on this thread! (Sorry it's been a
> > > while) I feel that we are very close to the final solution.
> > > >
> > > >
> > > >     Hey Jason and Mike, I have two quick questions on the new
> features
> > > here:
> > > >
> > > >       1.  so our proposal is that until we add a new static member
> into
> > > the group (scale up), we will not trigger rebalance until the
> > "registration
> > > timeout"( the member has been offline for too long)? How about leader's
> > > rejoin request, I think we should still trigger rebalance when that
> > > happens, since the consumer group may have new topics to consume?
> > > >       2.  I'm not very clear on the scale up scenario in static
> > > membership here. Should we fallback to dynamic membership while
> > > adding/removing hosts (by setting member.name = null), or we still
> want
> > > to add instances with `member.name` so that we eventually
> expand/shrink
> > > the static membership? I personally feel the easier solution is to spin
> > up
> > > new members and wait until either the same "registration timeout" or a
> > > "scale up timeout" before starting the rebalance. What do you think?
> > > >
> > > >     Meanwhile I will go ahead to make changes to the KIP with our
> newly
> > > discussed items and details. Really excited to see the design has
> become
> > > more solid.
> > > >
> > > >     Best,
> > > >     Boyang
> > > >
> > > >     ________________________________
> > > >     From: Jason Gustafson <ja...@confluent.io>
> > > >     Sent: Saturday, August 25, 2018 6:04 AM
> > > >     To: dev
> > > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> > > by specifying member id
> > > >
> > > >     Hey Mike,
> > > >
> > > >     Yeah, that's a good point. A long "registration timeout" may not
> be
> > > a great
> > > >     idea. Perhaps in practice you'd set it long enough to be able to
> > > detect a
> > > >     failure and provision a new instance. Maybe on the order of 10
> > > minutes is
> > > >     more reasonable.
> > > >
> > > >     In any case, it's probably a good idea to have an administrative
> > way
> > > to
> > > >     force deregistration. One option is to extend the DeleteGroups
> API
> > > with a
> > > >     list of members names.
> > > >
> > > >     -Jason
> > > >
> > > >
> > > >
> > > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > > mfreyberger@appnexus.com>
> > > >     wrote:
> > > >
> > > >     > Jason,
> > > >     >
> > > >     > Regarding step 4 in your proposal which suggests beginning a
> long
> > > timer
> > > >     > (30 minutes) when a static member leaves the group, would there
> > > also be the
> > > >     > ability for an admin to force a static membership expiration?
> > > >     >
> > > >     > I’m thinking that during particular types of outages or
> upgrades
> > > users
> > > >     > would want forcefully remove a static member from the group.
> > > >     >
> > > >     > So the user would shut the consumer down normally, which
> wouldn’t
> > > trigger
> > > >     > a rebalance. Then the user could use an admin CLI tool to force
> > > remove that
> > > >     > consumer from the group, so the TopicPartitions that were
> > > previously owned
> > > >     > by that consumer can be released.
> > > >     >
> > > >     > At a high level, we need consumer groups to gracefully handle
> > > intermittent
> > > >     > failures and permanent failures. Currently, the consumer group
> > > protocol
> > > >     > handles permanent failures well, but does not handle
> intermittent
> > > failures
> > > >     > well (it creates unnecessary rebalances). I want to make sure
> the
> > > overall
> > > >     > solution here handles both intermittent failures and permanent
> > > failures,
> > > >     > rather than sacrificing support for permanent failures in order
> > to
> > > provide
> > > >     > support for intermittent failures.
> > > >     >
> > > >     > Mike
> > > >     >
> > > >     > Sent from my iPhone
> > > >     >
> > > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> > jason@confluent.io>
> > > wrote:
> > > >     > >
> > > >     > > Hey Guozhang,
> > > >     > >
> > > >     > > Responses below:
> > > >     > >
> > > >     > > Originally I was trying to kill more birds with one stone
> with
> > > KIP-345,
> > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > shutting
> > > down a
> > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > email), and
> > > >     > >> hence proposing to have a pure static-membership protocol.
> But
> > > thinking
> > > >     > >> twice about it I now feel it may be too ambitious and worth
> > > fixing in
> > > >     > >> another KIP.
> > > >     > >
> > > >     > >
> > > >     > > I was considering an extension to support pre-initialization
> of
> > > the
> > > >     > static
> > > >     > > members of the group, but I agree we should probably leave
> this
> > > problem
> > > >     > for
> > > >     > > future work.
> > > >     > >
> > > >     > > 1. How this longish static member expiration timeout defined?
> > Is
> > > it via a
> > > >     > >> broker, hence global config, or via a client config which
> can
> > be
> > > >     > >> communicated to broker via JoinGroupRequest?
> > > >     > >
> > > >     > >
> > > >     > > I am not too sure. I tend to lean toward server-side configs
> > > because they
> > > >     > > are easier to evolve. If we have to add something to the
> > > protocol, then
> > > >     > > we'll be stuck with it forever.
> > > >     > >
> > > >     > > 2. Assuming that for static members, LEAVE_GROUP request will
> > not
> > > >     > trigger a
> > > >     > >> rebalance immediately either, similar to session timeout,
> but
> > > only the
> > > >     > >> longer member expiration timeout, can we remove the
> internal "
> > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > walk-around
> > > >     > then?
> > > >     > >
> > > >     > >
> > > >     > > Yeah, I hope we can ultimately get rid of it, but we may need
> > it
> > > for
> > > >     > > compatibility with older brokers. A related question is what
> > > should be
> > > >     > the
> > > >     > > behavior of the consumer if `member.name` is provided but
> the
> > > broker
> > > >     > does
> > > >     > > not support it? We could either fail or silently downgrade to
> > > dynamic
> > > >     > > membership.
> > > >     > >
> > > >     > > -Jason
> > > >     > >
> > > >     > >
> > > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > >     > wrote:
> > > >     > >>
> > > >     > >> Hey Jason,
> > > >     > >>
> > > >     > >> I like your idea to simplify the upgrade protocol to allow
> > > co-exist of
> > > >     > >> static and dynamic members. Admittedly it may make the
> > > coordinator-side
> > > >     > >> logic a bit more complex, but I think it worth doing it.
> > > >     > >>
> > > >     > >> Originally I was trying to kill more birds with one stone
> with
> > > KIP-345,
> > > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> > shutting
> > > down a
> > > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > > email), and
> > > >     > >> hence proposing to have a pure static-membership protocol.
> But
> > > thinking
> > > >     > >> twice about it I now feel it may be too ambitious and worth
> > > fixing in
> > > >     > >> another KIP. With that, I think what you've proposed here
> is a
> > > good way
> > > >     > to
> > > >     > >> go for KIP-345 itself.
> > > >     > >>
> > > >     > >> Note there are a few details in your proposal we'd still
> need
> > > to figure
> > > >     > >> out:
> > > >     > >>
> > > >     > >> 1. How this longish static member expiration timeout
> defined?
> > > Is it via
> > > >     > a
> > > >     > >> broker, hence global config, or via a client config which
> can
> > be
> > > >     > >> communicated to broker via JoinGroupRequest?
> > > >     > >>
> > > >     > >> 2. Assuming that for static members, LEAVE_GROUP request
> will
> > > not
> > > >     > trigger a
> > > >     > >> rebalance immediately either, similar to session timeout,
> but
> > > only the
> > > >     > >> longer member expiration timeout, can we remove the
> internal "
> > > >     > >> internal.leave.group.on.close" config, which is a quick
> > > walk-around
> > > >     > then?
> > > >     > >>
> > > >     > >>
> > > >     > >>
> > > >     > >> Guozhang
> > > >     > >>
> > > >     > >>
> > > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > > jason@confluent.io>
> > > >     > >> wrote:
> > > >     > >>
> > > >     > >>> Hey All,
> > > >     > >>>
> > > >     > >>> Nice to see some solid progress on this. It sounds like one
> > of
> > > the
> > > >     > >>> complications is allowing static and dynamic registration
> to
> > > coexist.
> > > >     > I'm
> > > >     > >>> wondering if we can do something like the following:
> > > >     > >>>
> > > >     > >>> 1. Statically registered members (those joining the group
> > with
> > > a
> > > >     > >> non-null `
> > > >     > >>> member.name`) maintain a session with the coordinator just
> > > like
> > > >     > dynamic
> > > >     > >>> members.
> > > >     > >>> 2. If a session is active for a static member when a
> > rebalance
> > > begins,
> > > >     > >> then
> > > >     > >>> basically we'll keep the current behavior. The rebalance
> will
> > > await the
> > > >     > >>> static member joining the group.
> > > >     > >>> 3. If a static member does not have an active session, then
> > the
> > > >     > >> coordinator
> > > >     > >>> will not wait for it to join, but will still include it in
> > the
> > > >     > rebalance.
> > > >     > >>> The coordinator will forward the cached subscription
> > > information to the
> > > >     > >>> leader and will cache the assignment after the rebalance
> > > completes.
> > > >     > (Note
> > > >     > >>> that we still have the generationId to fence offset commits
> > > from a
> > > >     > static
> > > >     > >>> zombie if the assignment changes.)
> > > >     > >>> 4. When a static member leaves the group or has its session
> > > expire, no
> > > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > > expire the
> > > >     > >> static
> > > >     > >>> registration. This would be a longish timeout (like 30
> > minutes
> > > say).
> > > >     > >>>
> > > >     > >>> So basically static members participate in all rebalances
> > > regardless
> > > >     > >>> whether they have an active session. In a given rebalance,
> > > some of the
> > > >     > >>> members may be static and some dynamic. The group leader
> can
> > > >     > >> differentiate
> > > >     > >>> the two based on the presence of the `member.name` (we
> have
> > > to add
> > > >     > this
> > > >     > >> to
> > > >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> > > leaders
> > > >     > >>> preferentially from the active members that support the
> > latest
> > > >     > JoinGroup
> > > >     > >>> protocol and are using static membership. If we have to
> > choose
> > > a leader
> > > >     > >>> with an old version, however, it would see all members in
> the
> > > group
> > > >     > >> (static
> > > >     > >>> or dynamic) as dynamic members and perform the assignment
> as
> > > usual.
> > > >     > >>>
> > > >     > >>> Would that work?
> > > >     > >>>
> > > >     > >>> -Jason
> > > >     > >>>
> > > >     > >>>
> > > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > > wangguoz@gmail.com>
> > > >     > >> wrote:
> > > >     > >>>
> > > >     > >>>> Hello Boyang,
> > > >     > >>>>
> > > >     > >>>> Thanks for the updated proposal, a few questions:
> > > >     > >>>>
> > > >     > >>>> 1. Where will "change-group-timeout" be communicated to
> the
> > > broker?
> > > >     > >> Will
> > > >     > >>>> that be a new field in the JoinGroupRequest, or are we
> going
> > > to
> > > >     > >>> piggy-back
> > > >     > >>>> on the existing session-timeout field (assuming that the
> > > original
> > > >     > value
> > > >     > >>>> will not be used anywhere in the static membership any
> > more)?
> > > >     > >>>>
> > > >     > >>>> 2. "However, if the consumer takes longer than session
> > > timeout to
> > > >     > >> return,
> > > >     > >>>> we shall still trigger rebalance but it could still try to
> > > catch
> > > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> > your
> > > proposal
> > > >     > >> is
> > > >     > >>>> that for static memberships, the broker will NOT trigger
> > > rebalance
> > > >     > even
> > > >     > >>>> after session-timeout has been detected, but only that
> after
> > > >     > >>>> change-group-timeout
> > > >     > >>>> which is supposed to be longer than session-timeout to be
> > > defined?
> > > >     > >>>>
> > > >     > >>>> 3. "A join group request with member.name set will be
> > > treated as
> > > >     > >>>> `static-membership` strategy", in this case, how would the
> > > switch from
> > > >     > >>>> dynamic to static happen, since whoever changed the
> > > member.name to
> > > >     > >>>> not-null
> > > >     > >>>> will be rejected, right?
> > > >     > >>>>
> > > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > > timeout to
> > > >     > >>> trigger
> > > >     > >>>> rebalance should be sufficient." this is also a bit
> unclear
> > > to me: who
> > > >     > >>> will
> > > >     > >>>> erase the cached mapping? Since it is on the broker-side I
> > > assume that
> > > >     > >>>> broker has to do it. Are you suggesting to use a new
> request
> > > for it?
> > > >     > >>>>
> > > >     > >>>> 5. "Halfway switch": following 3) above, if your proposal
> is
> > > basically
> > > >     > >> to
> > > >     > >>>> let "first join-request wins", and the strategy will stay
> as
> > > is until
> > > >     > >> all
> > > >     > >>>> members are gone, then this will also not happen since
> > > whoever used
> > > >     > >>>> different strategy as the first guy who sends join-group
> > > request will
> > > >     > >> be
> > > >     > >>>> rejected right?
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> Guozhang
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > > john@confluent.io>
> > > >     > >> wrote:
> > > >     > >>>>
> > > >     > >>>>> This sounds good to me!
> > > >     > >>>>>
> > > >     > >>>>> Thanks for the time you've spent on it,
> > > >     > >>>>> -John
> > > >     > >>>>>
> > > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > > bchen11@outlook.com>
> > > >     > >>>> wrote:
> > > >     > >>>>>
> > > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently
> > and
> > > >     > >> haven't
> > > >     > >>>> got
> > > >     > >>>>>> time to update this thread. To summarize what we come up
> > so
> > > far,
> > > >     > >> here
> > > >     > >>>> is
> > > >     > >>>>> a
> > > >     > >>>>>> draft updated plan:
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Introduce a new config called `member.name` which is
> > > supposed to
> > > >     > >> be
> > > >     > >>>>>> provided uniquely by the consumer client. The broker
> will
> > > maintain
> > > >     > >> a
> > > >     > >>>>> cache
> > > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > > request with
> > > >     > >>>>>> member.name set will be treated as `static-membership`
> > > strategy,
> > > >     > >> and
> > > >     > >>>>> will
> > > >     > >>>>>> reject any join group request without member.name. So
> > this
> > > >     > >>>> coordination
> > > >     > >>>>>> change will be differentiated from the
> > `dynamic-membership`
> > > >     > >> protocol
> > > >     > >>> we
> > > >     > >>>>>> currently have.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When handling static join group request:
> > > >     > >>>>>>
> > > >     > >>>>>>  1.   The broker will check the membership to see
> whether
> > > this is
> > > >     > >> a
> > > >     > >>>> new
> > > >     > >>>>>> member. If new, broker allocate a unique member id,
> cache
> > > the
> > > >     > >> mapping
> > > >     > >>>> and
> > > >     > >>>>>> move to rebalance stage.
> > > >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> > > will not
> > > >     > >>>> change
> > > >     > >>>>>> group state, and return its cached member.id and
> current
> > > >     > >> assignment.
> > > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin
> with
> > > pair
> > > >     > >>> member
> > > >     > >>>>>> name and id, I think for join group request it is ok to
> > > leave
> > > >     > >> member
> > > >     > >>> id
> > > >     > >>>>>> blank as member name is the unique identifier. In commit
> > > offset
> > > >     > >>> request
> > > >     > >>>>> we
> > > >     > >>>>>> *must* have both.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When handling commit offset request, if enabled with
> > static
> > > >     > >>> membership,
> > > >     > >>>>>> each time the commit request must have both member.name
> > and
> > > >     > >>> member.id
> > > >     > >>>> to
> > > >     > >>>>>> be identified as a `certificated member`. If not, this
> > > means there
> > > >     > >>> are
> > > >     > >>>>>> duplicate consumer members with same member name and the
> > > request
> > > >     > >> will
> > > >     > >>>> be
> > > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> When rolling restart/shutting down gracefully, the
> client
> > > will
> > > >     > >> send a
> > > >     > >>>>>> leave group request (static membership mode). In static
> > > membership,
> > > >     > >>> we
> > > >     > >>>>> will
> > > >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> > > provided by
> > > >     > >>>>> leader.
> > > >     > >>>>>> So we will wait for all the members to rejoin the group
> > and
> > > do
> > > >     > >>> exactly
> > > >     > >>>>> one
> > > >     > >>>>>> rebalance since all members are expected to rejoin
> within
> > > timeout.
> > > >     > >> If
> > > >     > >>>>>> consumer crashes, the join group request from the
> > restarted
> > > >     > >> consumer
> > > >     > >>>> will
> > > >     > >>>>>> be recognized as an existing member and be handled as
> > above
> > > >     > >> condition
> > > >     > >>>> 1;
> > > >     > >>>>>> However, if the consumer takes longer than session
> timeout
> > > to
> > > >     > >> return,
> > > >     > >>>> we
> > > >     > >>>>>> shall still trigger rebalance but it could still try to
> > > catch
> > > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > > timeout, its
> > > >     > >>>> cached
> > > >     > >>>>>> state on broker will be garbage collected and trigger a
> > new
> > > >     > >> rebalance
> > > >     > >>>>> when
> > > >     > >>>>>> it finally joins.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> And consider the switch between dynamic to static
> > > membership.
> > > >     > >>>>>>
> > > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise
> the
> > > >     > >> membership
> > > >     > >>>> to
> > > >     > >>>>>> static and wait for all the current members to restart,
> > > since their
> > > >     > >>>>>> membership is still dynamic. Here our assumption is that
> > the
> > > >     > >> restart
> > > >     > >>>>>> process shouldn't take a long time, as long restart is
> > > breaking the
> > > >     > >>>>>> `rebalance timeout` in whatever membership protocol we
> are
> > > using.
> > > >     > >>>> Before
> > > >     > >>>>>> restart, all dynamic member join requests will be
> > rejected.
> > > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> > which
> > > should
> > > >     > >>> be
> > > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> > session
> > > timeout
> > > >     > >>> to
> > > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> > current
> > > >     > >>> behavior)
> > > >     > >>>>>>  3.  Halfway switch: a corner case is like some clients
> > keep
> > > >     > >> dynamic
> > > >     > >>>>>> membership while some keep static membership. This will
> > > cause the
> > > >     > >>> group
> > > >     > >>>>>> rebalance forever without progress because
> dynamic/static
> > > states
> > > >     > >> are
> > > >     > >>>>>> bouncing each other. This could guarantee that we will
> not
> > > make the
> > > >     > >>>>>> consumer group work in a wrong state by having half
> static
> > > and half
> > > >     > >>>>> dynamic.
> > > >     > >>>>>>
> > > >     > >>>>>> To guarantee correctness, we will also push the member
> > > name/id pair
> > > >     > >>> to
> > > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > > upgrade the
> > > >     > >> API
> > > >     > >>>>>> version, these details will be further discussed back in
> > > the KIP.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Are there any concern for this high level proposal? Just
> > > want to
> > > >     > >>>>> reiterate
> > > >     > >>>>>> on the core idea of the KIP: "If the broker recognize
> this
> > > consumer
> > > >     > >>> as
> > > >     > >>>> an
> > > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > > >     > >>>>>>
> > > >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal
> is
> > > much
> > > >     > >> more
> > > >     > >>>>>> robust than previous one!
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> Best,
> > > >     > >>>>>>
> > > >     > >>>>>> Boyang
> > > >     > >>>>>>
> > > >     > >>>>>> ________________________________
> > > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > > >     > >>>>>> To: dev@kafka.apache.org
> > > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > > rebalances
> > > >     > >>> by
> > > >     > >>>>>> specifying member id
> > > >     > >>>>>>
> > > >     > >>>>>> Hi,
> > > >     > >>>>>>
> > > >     > >>>>>> thanks for the detailed discussion. I learned a lot
> about
> > > internals
> > > >     > >>>> again
> > > >     > >>>>>> :)
> > > >     > >>>>>>
> > > >     > >>>>>> I like the idea or a user config `member.name` and to
> > keep
> > > `
> > > >     > >>> member.id`
> > > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> > client.id`
> > > might
> > > >     > >>> not
> > > >     > >>>>>> be a good idea.
> > > >     > >>>>>>
> > > >     > >>>>>> To clarify the algorithm, each time we generate a new `
> > > member.id`,
> > > >     > >>> we
> > > >     > >>>>>> also need to update the "group membership" information
> > (ie,
> > > mapping
> > > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id
> `
> > > replaces
> > > >     > >>> the
> > > >     > >>>>>> old entry in the cache.
> > > >     > >>>>>>
> > > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > > member.id`
> > > >     > >>>> mapping
> > > >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> > > this IMHO.
> > > >     > >>>>>>
> > > >     > >>>>>> For changing the default value of config
> > > `leave.group.on.close`. I
> > > >     > >>>> agree
> > > >     > >>>>>> with John, that we should not change the default config,
> > > because it
> > > >     > >>>>>> would impact all consumer groups with dynamic
> assignment.
> > > However,
> > > >     > >> I
> > > >     > >>>>>> think we can document, that if static assignment is used
> > > (ie,
> > > >     > >>>>>> `member.name` is configured) we never send a
> > > LeaveGroupRequest
> > > >     > >>>>>> regardless of the config. Note, that the config is
> > > internal, so not
> > > >     > >>>> sure
> > > >     > >>>>>> how to document this in detail. We should not expose the
> > > internal
> > > >     > >>>> config
> > > >     > >>>>>> in the docs.
> > > >     > >>>>>>
> > > >     > >>>>>> About upgrading: why do we need have two rolling bounces
> > > and encode
> > > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > > >     > >>>>>>
> > > >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> > > static, I
> > > >     > >>>> don't
> > > >     > >>>>>> see any reason why both should not work together and
> > single
> > > rolling
> > > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > > consumer and
> > > >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> > > and the
> > > >     > >>> broker
> > > >     > >>>>>> registers the [member.name, member.id] in the cache.
> Why
> > > would
> > > >     > >> this
> > > >     > >>>>>> interfere with all other consumer that use dynamic
> > > assignment?
> > > >     > >>>>>>
> > > >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> > > need to
> > > >     > >> check
> > > >     > >>>> if
> > > >     > >>>>>> the mapping [member.name, member.id] contains the send
> `
> > > member.id`
> > > >     > >>> --
> > > >     > >>>> I
> > > >     > >>>>>> don't think this is necessary -- it seems to be
> sufficient
> > > to check
> > > >     > >>> the
> > > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as
> > be
> > > do
> > > >     > >> today
> > > >     > >>> --
> > > >     > >>>>>> thus, checking `member.id` does not require any change
> > > IMHO.
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>> -Matthias
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > > >     > >>>>>>> @James
> > > >     > >>>>>>>
> > > >     > >>>>>>> What you described is true: the transition from dynamic
> > to
> > > static
> > > >     > >>>>>>> memberships are not thought through yet. But I do not
> > > think it is
> > > >     > >>> an
> > > >     > >>>>>>> impossible problem: note that we indeed moved the
> offset
> > > commit
> > > >     > >>> from
> > > >     > >>>> ZK
> > > >     > >>>>>> to
> > > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> > > first to
> > > >     > >>>>>>> double-commits on both zk and coordinator, and then do
> a
> > > second
> > > >     > >>> round
> > > >     > >>>>> to
> > > >     > >>>>>>> turn the zk off.
> > > >     > >>>>>>>
> > > >     > >>>>>>> So just to throw a wild idea here: also following a
> > > >     > >>>> two-rolling-bounce
> > > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> > > "static"
> > > >     > >>> while
> > > >     > >>>>>> keep
> > > >     > >>>>>>> the registry-id field empty still, in this case, the
> > > coordinator
> > > >     > >>>> still
> > > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> > while
> > > >     > >>> allowing
> > > >     > >>>>> the
> > > >     > >>>>>>> protocol to be set to "static"; after the first rolling
> > > bounce,
> > > >     > >> the
> > > >     > >>>>> group
> > > >     > >>>>>>> protocol is already "static", then a second rolling
> > bounce
> > > is
> > > >     > >>>> triggered
> > > >     > >>>>>> and
> > > >     > >>>>>>> this time we set the registry-id.
> > > >     > >>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>> Guozhang
> > > >     > >>>>>>>
> > > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > > >     > >> wushujames@gmail.com>
> > > >     > >>>>>> wrote:
> > > >     > >>>>>>>
> > > >     > >>>>>>>> Guozhang, in a previous message, you proposed said
> this:
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > > wangguoz@gmail.com
> > > >     > >>>
> > > >     > >>>>> wrote:
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > > fields:
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic"
> membership
> > > >     > >>> protocols.
> > > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > > pre-defined
> > > >     > >>> member
> > > >     > >>>>> id.
> > > >     > >>>>>>>>> 1.c) with "static" membership, we also add an
> optional
> > > >     > >>>>>>>>> "group-change-timeout" value.
> > > >     > >>>>>>>>>
> > > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> > > protocols
> > > >     > >>> for
> > > >     > >>>>> all
> > > >     > >>>>>>>>> group members: we accept the protocol on the first
> > joined
> > > >     > >> member
> > > >     > >>> of
> > > >     > >>>>> the
> > > >     > >>>>>>>>> group, and if later joining members indicate a
> > different
> > > >     > >>> membership
> > > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > > value was
> > > >     > >>>>> different
> > > >     > >>>>>>>> to
> > > >     > >>>>>>>>> the first joined member, we reject it as well.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> What will happen if we have an already-deployed
> > > application that
> > > >     > >>>> wants
> > > >     > >>>>>> to
> > > >     > >>>>>>>> switch to using static membership? Let’s say there are
> > 10
> > > >     > >>> instances
> > > >     > >>>> of
> > > >     > >>>>>> it.
> > > >     > >>>>>>>> As the instances go through a rolling restart, they
> will
> > > switch
> > > >     > >>> from
> > > >     > >>>>>>>> dynamic membership (the default?) to static
> membership.
> > > As each
> > > >     > >>> one
> > > >     > >>>>>> leaves
> > > >     > >>>>>>>> the group and restarts, they will be rejected from the
> > > group
> > > >     > >>>> (because
> > > >     > >>>>>> the
> > > >     > >>>>>>>> group is currently using dynamic membership). The
> group
> > > will
> > > >     > >>> shrink
> > > >     > >>>>> down
> > > >     > >>>>>>>> until there is 1 node handling all the traffic. After
> > > that one
> > > >     > >>>>> restarts,
> > > >     > >>>>>>>> the group will switch over to static membership.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> Is that right? That means that the transition plan
> from
> > > dynamic
> > > >     > >> to
> > > >     > >>>>>> static
> > > >     > >>>>>>>> membership isn’t very smooth.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> I’m not really sure what can be done in this case.
> This
> > > reminds
> > > >     > >> me
> > > >     > >>>> of
> > > >     > >>>>>> the
> > > >     > >>>>>>>> transition plans that were discussed for moving from
> > > >     > >>> zookeeper-based
> > > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That
> was
> > > also
> > > >     > >>> hard,
> > > >     > >>>>> and
> > > >     > >>>>>>>> ultimately we decided not to build that.
> > > >     > >>>>>>>>
> > > >     > >>>>>>>> -James
> > > >     > >>>>>>>>
> > > >     > >>>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>>
> > > >     > >>>>>
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>>
> > > >     > >>>> --
> > > >     > >>>> -- Guozhang
> > > >     > >>>>
> > > >     > >>>
> > > >     > >>
> > > >     > >>
> > > >     > >>
> > > >     > >> --
> > > >     > >> -- Guozhang
> > > >     > >>
> > > >     >
> > > >
> > > >
> > >
> > >
> >
> > --
> > -Regards,
> > Mayuresh R. Gharat
> > (862) 250-7125
> >
>
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Mayuresh for the feedback! Do you have a quick example for passing in consumer config dynamically? I mainly use Kafka Streams at my daily work so probably missing the idea how to do it in the current consumer setting.


For differentiating session timeout and registration timeout, I would try to enhance the documentation in the first stage to see how people react to the confusion (would be great if they feel straightforward!). Since one doesn't have to fully understand the difference unless defining the new config "member name", for current users we could buy some time to listen to their understandings and improve our documentation correspondingly in the follow-up KIPs.


Boyang

________________________________
From: Mayuresh Gharat <gh...@gmail.com>
Sent: Sunday, November 11, 2018 1:06 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

Thanks for the reply.

Please find the replies inline below :
For having a consumer config at runtime, I think it's not necessary to
address in this KIP because most companies run sidecar jobs through daemon
software like puppet. It should be easy to change the config through script
or UI without actual code change. We still want to leave flexibility for
user to define member name as they like.
---- This might be little different for companies that use configuration
management tools that does not allow the applications to define/change the
configs dynamically. For example, if we use something similar to spring to
pull in the configs for the KafkaConsumer and pass it to the constructor to
create the KafkaConsumer object, it will be hard to specify a unique value
to the "MEMBER_NAME" config unless someone deploying the app generates a
unique string for this config outside the deployment workflow and copies it
statically before starting up each consumer instance. Unless we can loosen
the criteria for uniqueness of this config value, for each consumer
instance in the consumer group, I am not sure of a better way of
addressing this. If we don't want to loosen the criteria, then providing a
dynamic way to pass this in at runtime, would put the onus of having the
same unique value each time a consumer is restarted, on to the application
that is running the consumer.

I just updated the kip about having both "registration timeout" and
"session timeout". The benefit of having two configs instead of one is to
reduce the mental burden for operation, for example user just needs to
unset "member name" to cast back to dynamic membership without worrying
about tuning the "session timeout" back to a smaller value.
--- That is a good point. I was thinking, if both the configs are
specified, it would be confusing for the end user without understanding the
internals of the consumer and its interaction with group coordinator, as
which takes precedence when and how it affects the consumer behavior. Just
my 2 cents.

Thanks,

Mayuresh

On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:

> Hey Mayuresh,
>
>
> thanks for the thoughtful questions! Let me try to answer your questions
> one by one.
>
>
> For having a consumer config at runtime, I think it's not necessary to
> address in this KIP because most companies run sidecar jobs through daemon
> software like puppet. It should be easy to change the config through script
> or UI without actual code change. We still want to leave flexibility for
> user to define member name as they like.
>
>
> I just updated the kip about having both "registration timeout" and
> "session timeout". The benefit of having two configs instead of one is to
> reduce the mental burden for operation, for example user just needs to
> unset "member name" to cast back to dynamic membership without worrying
> about tuning the "session timeout" back to a smaller value.
>
>
> For backup topic, I think it's a low-level detail which could be addressed
> in the implementation. I feel no preference of adding a new topic vs reuse
> consumer offsets topic. I will do more analysis and make a trade-off
> comparison. Nice catch!
>
>
> I hope the explanations make sense to you. I will keep polishing on the
> edge cases and details.
>
>
> Best,
>
> Boyang
>
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Saturday, November 10, 2018 10:25 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for the KIP and sorry for being late to the party. This KIP is
> really useful for us at Linkedin.
>
> I had a few questions :
>
> The idea of having static member name seems nice, but instead of a config,
> would it be possible for it to be passed in to the consumer at runtime?
> This is because an app might want to decide the config value at runtime
> using its host information for example, to generate the unique member name.
>
> Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> wondering if we can reuse the session timeout here. This might help us to
> have one less config on the consumer.
>
> The KIP also talks about adding another internal topic "static_member_map".
> Would the semantics (GroupCoordinator broker, topic configs) be the same as
> __consumer_offsets topic?
>
> Thanks,
>
> Mayuresh
>
>
> On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > I took a quick pass of the proposal. First I would say it's a very
> > brilliant initiative from Konstantine and Confluent folks. To draft up a
> > proposal like this needs deep understanding of the rebalance protocol! I
> > summarized some thoughts here.
> >
> >
> > Overall the motivations of the two proposals align on that:
> >
> >   1.  Both believe the invariant resource (belonging to the same process)
> > should be preserved across rebalance.
> >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > redistribution. I don't use rebalance here since part one of the
> > cooperative proposal could potentially introduce more rebalances but only
> > on must-move resources.
> >   3.  Scale up/down and rolling bounce are causing unnecessary resource
> > shuffling that need to be mitigated.
> >
> >
> > On motivation level, I think both approach could solve/mitigate the above
> > issues. They are just different in design philosophy, or I would say the
> > perspective difference between framework user and algorithm designer.
> >
> >
> > Two proposals have different focuses. KIP-345 is trying to place more
> > fine-grained control on the broker side to reduce the unnecessary
> > rebalances, while keeping the client logic intact. This is pretty
> intuitive
> > cause-effect for normal developers who are not very familiar with
> rebalance
> > protocol. As a developer working with Kafka Streams daily, I'd be happy
> to
> > see a simplified rebalance protocol and just focus on maintaining the
> > stream/consumer jobs. Too many rebalances raised my concern on the job
> > health. To be concise, static membership has the advantage of reducing
> > mental burden.
> >
> >
> > Cooperative proposal takes thoughtful approach on client side. We want to
> > have fine-grained control on the join/exit group behaviors and make the
> > current dynamic membership better to address above issues. I do feel our
> > idea crossed on the delayed rebalance when we scale up/down, which could
> > potentially reduce the state shuffling and decouple the behavior from
> > session timeout which is already overloaded.  In this sense, I believe
> both
> > approaches would serve well in making "reasonable rebalance" happen at
> the
> > "right timing".
> >
> >
> > However, based on my understanding, either 345 or cooperative rebalancing
> > is not solving the problem Mike has proposed: could we do a better job at
> > scaling up/down in ideal timing? My initial response was to introduce an
> > admin API which now I feel is sub-optimal, in that the goal of smooth
> > transition is to make sure the newly up hosts are actually "ready". For
> > example:
> >
> >
> > We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> > time we would like to scale up to 8 hosts, with the current improvements
> we
> > could reduce 4 potential rebalances to a single one. But the new hosts
> are
> > yet unknown to be "ready" if they need to reconstruct the local state. To
> > be actually ready, we need 4 standby tasks running on those empty hosts
> and
> > leader needs to wait for the signal of "replay/reconstruct complete" to
> > actually involve them into the main consumer group. Otherwise, rebalance
> > just kills our performance since we need to wait indefinite long for task
> > migration.
> >
> >
> > The scale down is also tricky such that we are not able to define a
> "true"
> > leave of a member. Rebalance immediately after "true" leaves are most
> > optimal comparing with human intervention. Does this make sense?
> >
> >
> > My intuition is that cooperative approach which was implemented on the
> > client side could better handle scaling cases than KIP 345, since it
> > involves a lot of algorithmic changes to define "replaying" stage, which
> I
> > feel would over-complicate broker logic if implemented on coordinator. If
> > we let 345 focus on reducing unnecessary rebalance, and let cooperative
> > approach focus on judging best timing of scale up/down, the two efforts
> > could be aligned. In long term, I feel the more complex improvement of
> > consumer protocol should happen on client side instead of server side
> which
> > is easier to test and has less global impact for the entire Kafka
> > production cluster.
> >
> >
> > Thanks again to Konstantine, Matthias and other folks in coming up with
> > this great client proposal. This is great complementation to KIP 345. In
> a
> > high level, we are not having any collision on the path and both
> proposals
> > are making sense here. Just need better sync to avoid duplicate effort :)
> >
> >
> > Best,
> >
> > Boyang
> >
> >
> > ________________________________
> > From: Boyang Chen <bc...@outlook.com>
> > Sent: Wednesday, November 7, 2018 1:57 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > deeper look and make a comparison between the two proposals.
> >
> >
> > Meanwhile for the scale down specifically for stateful streaming, we
> could
> > actually introduce a new status called "learner" where the newly up hosts
> > could try to catch up with the assigned task progress first before
> > triggering the rebalance, from which we don't see a sudden dip on the
> > progress. However, it is built on top of the success of KIP-345.
> >
> >
> > ________________________________
> > From: Matthias J. Sax <ma...@confluent.io>
> > Sent: Wednesday, November 7, 2018 7:02 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hey,
> >
> > there was quite a pause on this KIP discussion and in the mean time, a
> > new design for incremental cooporative rebalance was suggested:
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
Incremental Cooperative Rebalancing: Support and Policies ...<https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies>
cwiki.apache.org
Rebalancing between distributed application processes in Apache Kafka was enhanced considerably when it was decoupled as logic from Kafka brokers and was moved as responsibility to the clients and specifically to Kafka Consumer. This pattern has been working robustly for quite a while now and has ...



> Incremental Cooperative Rebalancing: Support and Policies ...<
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
> >
> cwiki.apache.org
> Rebalancing between distributed application processes in Apache Kafka was
> enhanced considerably when it was decoupled as logic from Kafka brokers and
> was moved as responsibility to the clients and specifically to Kafka
> Consumer. This pattern has been working robustly for quite a while now and
> has ...
>
>
>
> >
> >
> > We should make sure that the proposal and this KIP align to each other.
> > Thoughts?
> >
> >
> > -Matthias
> >
> > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > Hey Mike,
> > >
> > >
> > > thanks for the feedback, the two question are very thoughtful!
> > >
> > >
> > >> 1) I am a little confused about the distinction for the leader. If the
> > consumer node that was assigned leader does a bounce (goes down and
> quickly
> > comes up) to update application code, will a rebalance be triggered? I >
> do
> > not think a bounce of the leader should trigger a rebalance.
> > >
> > > For Q1 my intention was to minimize the change within one KIP, since
> the
> > leader rejoining case could be addressed separately.
> > >
> > >
> > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > gracefully increase the number of nodes in the cluster. I think we need
> to
> > support graceful shrink down as well. If I set the registration timeout
> to
> > 5 minutes > to handle rolling restarts or intermittent failures without
> > shuffling state, I don't want to wait 5 minutes in order for the group to
> > rebalance if I am intentionally removing a node from the cluster. I am
> not
> > sure the best way to > do this. One idea I had was adding the ability
> for a
> > CLI or Admin API to force a rebalance of the group. This would allow for
> an
> > admin to trigger the rebalance manually without waiting the entire
> > registration timeout on > shrink down. What do you think?
> > >
> > > For 2) my understanding is that for scaling down case it is better to
> be
> > addressed by CLI tool than code logic, since only by human evaluation we
> > could decide whether it is a "right timing" -- the time when all the
> > scaling down consumers are offline -- to kick in rebalance. Unless we
> > introduce another term on coordinator which indicates the target consumer
> > group size, broker will find it hard to decide when to start rebalance.
> So
> > far I prefer to hold the implementation for that, but agree we could
> > discuss whether we want to introduce admin API in this KIP or a separate
> > one.
> > >
> > >
> > > Thanks again for the proposed ideas!
> > >
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Mike Freyberger <mi...@xandr.com>
> > > Sent: Monday, November 5, 2018 6:13 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> > >
> > > Boyang,
> > >
> > > Thanks for updating the KIP. It's shaping up well. Two things:
> > >
> > > 1) I am a little confused about the distinction for the leader. If the
> > consumer node that was assigned leader does a bounce (goes down and
> quickly
> > comes up) to update application code, will a rebalance be triggered? I do
> > not think a bounce of the leader should trigger a rebalance.
> > >
> > > 2) The timeout for shrink up makes a lot of sense and allows to
> > gracefully increase the number of nodes in the cluster. I think we need
> to
> > support graceful shrink down as well. If I set the registration timeout
> to
> > 5 minutes to handle rolling restarts or intermittent failures without
> > shuffling state, I don't want to wait 5 minutes in order for the group to
> > rebalance if I am intentionally removing a node from the cluster. I am
> not
> > sure the best way to do this. One idea I had was adding the ability for a
> > CLI or Admin API to force a rebalance of the group. This would allow for
> an
> > admin to trigger the rebalance manually without waiting the entire
> > registration timeout on shrink down. What do you think?
> > >
> > > Mike
> > >
> > > On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > >
> > >     Btw, I updated KIP 345 based on my understanding. Feel free to take
> > another round of look:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > KIP-345: Introduce static membership protocol to reduce ...<
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > >
> > > cwiki.apache.org
> > > For stateful applications, one of the biggest performance bottleneck is
> > the state shuffling. In Kafka consumer, there is a concept called
> > "rebalance" which means that for given M partitions and N consumers in
> one
> > consumer group, Kafka will try to balance the load between consumers and
> > ideally have ...
> > >
> > >
> > >
> > >
> > >     KIP-345: Introduce static membership protocol to reduce ...<
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > >
> > >     cwiki.apache.org
> > >     For stateful applications, one of the biggest performance
> bottleneck
> > is the state shuffling. In Kafka consumer, there is a concept called
> > "rebalance" which means that for given M partitions and N consumers in
> one
> > consumer group, Kafka will try to balance the load between consumers and
> > ideally have ...
> > >
> > >
> > >
> > >
> > >
> > >     ________________________________
> > >     From: Boyang Chen <bc...@outlook.com>
> > >     Sent: Monday, October 29, 2018 12:34 PM
> > >     To: dev@kafka.apache.org
> > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by specifying member id
> > >
> > >     Thanks everyone for the input on this thread! (Sorry it's been a
> > while) I feel that we are very close to the final solution.
> > >
> > >
> > >     Hey Jason and Mike, I have two quick questions on the new features
> > here:
> > >
> > >       1.  so our proposal is that until we add a new static member into
> > the group (scale up), we will not trigger rebalance until the
> "registration
> > timeout"( the member has been offline for too long)? How about leader's
> > rejoin request, I think we should still trigger rebalance when that
> > happens, since the consumer group may have new topics to consume?
> > >       2.  I'm not very clear on the scale up scenario in static
> > membership here. Should we fallback to dynamic membership while
> > adding/removing hosts (by setting member.name = null), or we still want
> > to add instances with `member.name` so that we eventually expand/shrink
> > the static membership? I personally feel the easier solution is to spin
> up
> > new members and wait until either the same "registration timeout" or a
> > "scale up timeout" before starting the rebalance. What do you think?
> > >
> > >     Meanwhile I will go ahead to make changes to the KIP with our newly
> > discussed items and details. Really excited to see the design has become
> > more solid.
> > >
> > >     Best,
> > >     Boyang
> > >
> > >     ________________________________
> > >     From: Jason Gustafson <ja...@confluent.io>
> > >     Sent: Saturday, August 25, 2018 6:04 AM
> > >     To: dev
> > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by specifying member id
> > >
> > >     Hey Mike,
> > >
> > >     Yeah, that's a good point. A long "registration timeout" may not be
> > a great
> > >     idea. Perhaps in practice you'd set it long enough to be able to
> > detect a
> > >     failure and provision a new instance. Maybe on the order of 10
> > minutes is
> > >     more reasonable.
> > >
> > >     In any case, it's probably a good idea to have an administrative
> way
> > to
> > >     force deregistration. One option is to extend the DeleteGroups API
> > with a
> > >     list of members names.
> > >
> > >     -Jason
> > >
> > >
> > >
> > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > mfreyberger@appnexus.com>
> > >     wrote:
> > >
> > >     > Jason,
> > >     >
> > >     > Regarding step 4 in your proposal which suggests beginning a long
> > timer
> > >     > (30 minutes) when a static member leaves the group, would there
> > also be the
> > >     > ability for an admin to force a static membership expiration?
> > >     >
> > >     > I’m thinking that during particular types of outages or upgrades
> > users
> > >     > would want forcefully remove a static member from the group.
> > >     >
> > >     > So the user would shut the consumer down normally, which wouldn’t
> > trigger
> > >     > a rebalance. Then the user could use an admin CLI tool to force
> > remove that
> > >     > consumer from the group, so the TopicPartitions that were
> > previously owned
> > >     > by that consumer can be released.
> > >     >
> > >     > At a high level, we need consumer groups to gracefully handle
> > intermittent
> > >     > failures and permanent failures. Currently, the consumer group
> > protocol
> > >     > handles permanent failures well, but does not handle intermittent
> > failures
> > >     > well (it creates unnecessary rebalances). I want to make sure the
> > overall
> > >     > solution here handles both intermittent failures and permanent
> > failures,
> > >     > rather than sacrificing support for permanent failures in order
> to
> > provide
> > >     > support for intermittent failures.
> > >     >
> > >     > Mike
> > >     >
> > >     > Sent from my iPhone
> > >     >
> > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> jason@confluent.io>
> > wrote:
> > >     > >
> > >     > > Hey Guozhang,
> > >     > >
> > >     > > Responses below:
> > >     > >
> > >     > > Originally I was trying to kill more birds with one stone with
> > KIP-345,
> > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> shutting
> > down a
> > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > email), and
> > >     > >> hence proposing to have a pure static-membership protocol. But
> > thinking
> > >     > >> twice about it I now feel it may be too ambitious and worth
> > fixing in
> > >     > >> another KIP.
> > >     > >
> > >     > >
> > >     > > I was considering an extension to support pre-initialization of
> > the
> > >     > static
> > >     > > members of the group, but I agree we should probably leave this
> > problem
> > >     > for
> > >     > > future work.
> > >     > >
> > >     > > 1. How this longish static member expiration timeout defined?
> Is
> > it via a
> > >     > >> broker, hence global config, or via a client config which can
> be
> > >     > >> communicated to broker via JoinGroupRequest?
> > >     > >
> > >     > >
> > >     > > I am not too sure. I tend to lean toward server-side configs
> > because they
> > >     > > are easier to evolve. If we have to add something to the
> > protocol, then
> > >     > > we'll be stuck with it forever.
> > >     > >
> > >     > > 2. Assuming that for static members, LEAVE_GROUP request will
> not
> > >     > trigger a
> > >     > >> rebalance immediately either, similar to session timeout, but
> > only the
> > >     > >> longer member expiration timeout, can we remove the internal "
> > >     > >> internal.leave.group.on.close" config, which is a quick
> > walk-around
> > >     > then?
> > >     > >
> > >     > >
> > >     > > Yeah, I hope we can ultimately get rid of it, but we may need
> it
> > for
> > >     > > compatibility with older brokers. A related question is what
> > should be
> > >     > the
> > >     > > behavior of the consumer if `member.name` is provided but the
> > broker
> > >     > does
> > >     > > not support it? We could either fail or silently downgrade to
> > dynamic
> > >     > > membership.
> > >     > >
> > >     > > -Jason
> > >     > >
> > >     > >
> > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > wangguoz@gmail.com>
> > >     > wrote:
> > >     > >>
> > >     > >> Hey Jason,
> > >     > >>
> > >     > >> I like your idea to simplify the upgrade protocol to allow
> > co-exist of
> > >     > >> static and dynamic members. Admittedly it may make the
> > coordinator-side
> > >     > >> logic a bit more complex, but I think it worth doing it.
> > >     > >>
> > >     > >> Originally I was trying to kill more birds with one stone with
> > KIP-345,
> > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> shutting
> > down a
> > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > email), and
> > >     > >> hence proposing to have a pure static-membership protocol. But
> > thinking
> > >     > >> twice about it I now feel it may be too ambitious and worth
> > fixing in
> > >     > >> another KIP. With that, I think what you've proposed here is a
> > good way
> > >     > to
> > >     > >> go for KIP-345 itself.
> > >     > >>
> > >     > >> Note there are a few details in your proposal we'd still need
> > to figure
> > >     > >> out:
> > >     > >>
> > >     > >> 1. How this longish static member expiration timeout defined?
> > Is it via
> > >     > a
> > >     > >> broker, hence global config, or via a client config which can
> be
> > >     > >> communicated to broker via JoinGroupRequest?
> > >     > >>
> > >     > >> 2. Assuming that for static members, LEAVE_GROUP request will
> > not
> > >     > trigger a
> > >     > >> rebalance immediately either, similar to session timeout, but
> > only the
> > >     > >> longer member expiration timeout, can we remove the internal "
> > >     > >> internal.leave.group.on.close" config, which is a quick
> > walk-around
> > >     > then?
> > >     > >>
> > >     > >>
> > >     > >>
> > >     > >> Guozhang
> > >     > >>
> > >     > >>
> > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > jason@confluent.io>
> > >     > >> wrote:
> > >     > >>
> > >     > >>> Hey All,
> > >     > >>>
> > >     > >>> Nice to see some solid progress on this. It sounds like one
> of
> > the
> > >     > >>> complications is allowing static and dynamic registration to
> > coexist.
> > >     > I'm
> > >     > >>> wondering if we can do something like the following:
> > >     > >>>
> > >     > >>> 1. Statically registered members (those joining the group
> with
> > a
> > >     > >> non-null `
> > >     > >>> member.name`) maintain a session with the coordinator just
> > like
> > >     > dynamic
> > >     > >>> members.
> > >     > >>> 2. If a session is active for a static member when a
> rebalance
> > begins,
> > >     > >> then
> > >     > >>> basically we'll keep the current behavior. The rebalance will
> > await the
> > >     > >>> static member joining the group.
> > >     > >>> 3. If a static member does not have an active session, then
> the
> > >     > >> coordinator
> > >     > >>> will not wait for it to join, but will still include it in
> the
> > >     > rebalance.
> > >     > >>> The coordinator will forward the cached subscription
> > information to the
> > >     > >>> leader and will cache the assignment after the rebalance
> > completes.
> > >     > (Note
> > >     > >>> that we still have the generationId to fence offset commits
> > from a
> > >     > static
> > >     > >>> zombie if the assignment changes.)
> > >     > >>> 4. When a static member leaves the group or has its session
> > expire, no
> > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > expire the
> > >     > >> static
> > >     > >>> registration. This would be a longish timeout (like 30
> minutes
> > say).
> > >     > >>>
> > >     > >>> So basically static members participate in all rebalances
> > regardless
> > >     > >>> whether they have an active session. In a given rebalance,
> > some of the
> > >     > >>> members may be static and some dynamic. The group leader can
> > >     > >> differentiate
> > >     > >>> the two based on the presence of the `member.name` (we have
> > to add
> > >     > this
> > >     > >> to
> > >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> > leaders
> > >     > >>> preferentially from the active members that support the
> latest
> > >     > JoinGroup
> > >     > >>> protocol and are using static membership. If we have to
> choose
> > a leader
> > >     > >>> with an old version, however, it would see all members in the
> > group
> > >     > >> (static
> > >     > >>> or dynamic) as dynamic members and perform the assignment as
> > usual.
> > >     > >>>
> > >     > >>> Would that work?
> > >     > >>>
> > >     > >>> -Jason
> > >     > >>>
> > >     > >>>
> > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > wangguoz@gmail.com>
> > >     > >> wrote:
> > >     > >>>
> > >     > >>>> Hello Boyang,
> > >     > >>>>
> > >     > >>>> Thanks for the updated proposal, a few questions:
> > >     > >>>>
> > >     > >>>> 1. Where will "change-group-timeout" be communicated to the
> > broker?
> > >     > >> Will
> > >     > >>>> that be a new field in the JoinGroupRequest, or are we going
> > to
> > >     > >>> piggy-back
> > >     > >>>> on the existing session-timeout field (assuming that the
> > original
> > >     > value
> > >     > >>>> will not be used anywhere in the static membership any
> more)?
> > >     > >>>>
> > >     > >>>> 2. "However, if the consumer takes longer than session
> > timeout to
> > >     > >> return,
> > >     > >>>> we shall still trigger rebalance but it could still try to
> > catch
> > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> your
> > proposal
> > >     > >> is
> > >     > >>>> that for static memberships, the broker will NOT trigger
> > rebalance
> > >     > even
> > >     > >>>> after session-timeout has been detected, but only that after
> > >     > >>>> change-group-timeout
> > >     > >>>> which is supposed to be longer than session-timeout to be
> > defined?
> > >     > >>>>
> > >     > >>>> 3. "A join group request with member.name set will be
> > treated as
> > >     > >>>> `static-membership` strategy", in this case, how would the
> > switch from
> > >     > >>>> dynamic to static happen, since whoever changed the
> > member.name to
> > >     > >>>> not-null
> > >     > >>>> will be rejected, right?
> > >     > >>>>
> > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > timeout to
> > >     > >>> trigger
> > >     > >>>> rebalance should be sufficient." this is also a bit unclear
> > to me: who
> > >     > >>> will
> > >     > >>>> erase the cached mapping? Since it is on the broker-side I
> > assume that
> > >     > >>>> broker has to do it. Are you suggesting to use a new request
> > for it?
> > >     > >>>>
> > >     > >>>> 5. "Halfway switch": following 3) above, if your proposal is
> > basically
> > >     > >> to
> > >     > >>>> let "first join-request wins", and the strategy will stay as
> > is until
> > >     > >> all
> > >     > >>>> members are gone, then this will also not happen since
> > whoever used
> > >     > >>>> different strategy as the first guy who sends join-group
> > request will
> > >     > >> be
> > >     > >>>> rejected right?
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> Guozhang
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > john@confluent.io>
> > >     > >> wrote:
> > >     > >>>>
> > >     > >>>>> This sounds good to me!
> > >     > >>>>>
> > >     > >>>>> Thanks for the time you've spent on it,
> > >     > >>>>> -John
> > >     > >>>>>
> > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > bchen11@outlook.com>
> > >     > >>>> wrote:
> > >     > >>>>>
> > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently
> and
> > >     > >> haven't
> > >     > >>>> got
> > >     > >>>>>> time to update this thread. To summarize what we come up
> so
> > far,
> > >     > >> here
> > >     > >>>> is
> > >     > >>>>> a
> > >     > >>>>>> draft updated plan:
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Introduce a new config called `member.name` which is
> > supposed to
> > >     > >> be
> > >     > >>>>>> provided uniquely by the consumer client. The broker will
> > maintain
> > >     > >> a
> > >     > >>>>> cache
> > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > request with
> > >     > >>>>>> member.name set will be treated as `static-membership`
> > strategy,
> > >     > >> and
> > >     > >>>>> will
> > >     > >>>>>> reject any join group request without member.name. So
> this
> > >     > >>>> coordination
> > >     > >>>>>> change will be differentiated from the
> `dynamic-membership`
> > >     > >> protocol
> > >     > >>> we
> > >     > >>>>>> currently have.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When handling static join group request:
> > >     > >>>>>>
> > >     > >>>>>>  1.   The broker will check the membership to see whether
> > this is
> > >     > >> a
> > >     > >>>> new
> > >     > >>>>>> member. If new, broker allocate a unique member id, cache
> > the
> > >     > >> mapping
> > >     > >>>> and
> > >     > >>>>>> move to rebalance stage.
> > >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> > will not
> > >     > >>>> change
> > >     > >>>>>> group state, and return its cached member.id and current
> > >     > >> assignment.
> > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with
> > pair
> > >     > >>> member
> > >     > >>>>>> name and id, I think for join group request it is ok to
> > leave
> > >     > >> member
> > >     > >>> id
> > >     > >>>>>> blank as member name is the unique identifier. In commit
> > offset
> > >     > >>> request
> > >     > >>>>> we
> > >     > >>>>>> *must* have both.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When handling commit offset request, if enabled with
> static
> > >     > >>> membership,
> > >     > >>>>>> each time the commit request must have both member.name
> and
> > >     > >>> member.id
> > >     > >>>> to
> > >     > >>>>>> be identified as a `certificated member`. If not, this
> > means there
> > >     > >>> are
> > >     > >>>>>> duplicate consumer members with same member name and the
> > request
> > >     > >> will
> > >     > >>>> be
> > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When rolling restart/shutting down gracefully, the client
> > will
> > >     > >> send a
> > >     > >>>>>> leave group request (static membership mode). In static
> > membership,
> > >     > >>> we
> > >     > >>>>> will
> > >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> > provided by
> > >     > >>>>> leader.
> > >     > >>>>>> So we will wait for all the members to rejoin the group
> and
> > do
> > >     > >>> exactly
> > >     > >>>>> one
> > >     > >>>>>> rebalance since all members are expected to rejoin within
> > timeout.
> > >     > >> If
> > >     > >>>>>> consumer crashes, the join group request from the
> restarted
> > >     > >> consumer
> > >     > >>>> will
> > >     > >>>>>> be recognized as an existing member and be handled as
> above
> > >     > >> condition
> > >     > >>>> 1;
> > >     > >>>>>> However, if the consumer takes longer than session timeout
> > to
> > >     > >> return,
> > >     > >>>> we
> > >     > >>>>>> shall still trigger rebalance but it could still try to
> > catch
> > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > timeout, its
> > >     > >>>> cached
> > >     > >>>>>> state on broker will be garbage collected and trigger a
> new
> > >     > >> rebalance
> > >     > >>>>> when
> > >     > >>>>>> it finally joins.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> And consider the switch between dynamic to static
> > membership.
> > >     > >>>>>>
> > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
> > >     > >> membership
> > >     > >>>> to
> > >     > >>>>>> static and wait for all the current members to restart,
> > since their
> > >     > >>>>>> membership is still dynamic. Here our assumption is that
> the
> > >     > >> restart
> > >     > >>>>>> process shouldn't take a long time, as long restart is
> > breaking the
> > >     > >>>>>> `rebalance timeout` in whatever membership protocol we are
> > using.
> > >     > >>>> Before
> > >     > >>>>>> restart, all dynamic member join requests will be
> rejected.
> > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> which
> > should
> > >     > >>> be
> > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> session
> > timeout
> > >     > >>> to
> > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> current
> > >     > >>> behavior)
> > >     > >>>>>>  3.  Halfway switch: a corner case is like some clients
> keep
> > >     > >> dynamic
> > >     > >>>>>> membership while some keep static membership. This will
> > cause the
> > >     > >>> group
> > >     > >>>>>> rebalance forever without progress because dynamic/static
> > states
> > >     > >> are
> > >     > >>>>>> bouncing each other. This could guarantee that we will not
> > make the
> > >     > >>>>>> consumer group work in a wrong state by having half static
> > and half
> > >     > >>>>> dynamic.
> > >     > >>>>>>
> > >     > >>>>>> To guarantee correctness, we will also push the member
> > name/id pair
> > >     > >>> to
> > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > upgrade the
> > >     > >> API
> > >     > >>>>>> version, these details will be further discussed back in
> > the KIP.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Are there any concern for this high level proposal? Just
> > want to
> > >     > >>>>> reiterate
> > >     > >>>>>> on the core idea of the KIP: "If the broker recognize this
> > consumer
> > >     > >>> as
> > >     > >>>> an
> > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > >     > >>>>>>
> > >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is
> > much
> > >     > >> more
> > >     > >>>>>> robust than previous one!
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Best,
> > >     > >>>>>>
> > >     > >>>>>> Boyang
> > >     > >>>>>>
> > >     > >>>>>> ________________________________
> > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > >     > >>>>>> To: dev@kafka.apache.org
> > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > >     > >>> by
> > >     > >>>>>> specifying member id
> > >     > >>>>>>
> > >     > >>>>>> Hi,
> > >     > >>>>>>
> > >     > >>>>>> thanks for the detailed discussion. I learned a lot about
> > internals
> > >     > >>>> again
> > >     > >>>>>> :)
> > >     > >>>>>>
> > >     > >>>>>> I like the idea or a user config `member.name` and to
> keep
> > `
> > >     > >>> member.id`
> > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> client.id`
> > might
> > >     > >>> not
> > >     > >>>>>> be a good idea.
> > >     > >>>>>>
> > >     > >>>>>> To clarify the algorithm, each time we generate a new `
> > member.id`,
> > >     > >>> we
> > >     > >>>>>> also need to update the "group membership" information
> (ie,
> > mapping
> > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id`
> > replaces
> > >     > >>> the
> > >     > >>>>>> old entry in the cache.
> > >     > >>>>>>
> > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > member.id`
> > >     > >>>> mapping
> > >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> > this IMHO.
> > >     > >>>>>>
> > >     > >>>>>> For changing the default value of config
> > `leave.group.on.close`. I
> > >     > >>>> agree
> > >     > >>>>>> with John, that we should not change the default config,
> > because it
> > >     > >>>>>> would impact all consumer groups with dynamic assignment.
> > However,
> > >     > >> I
> > >     > >>>>>> think we can document, that if static assignment is used
> > (ie,
> > >     > >>>>>> `member.name` is configured) we never send a
> > LeaveGroupRequest
> > >     > >>>>>> regardless of the config. Note, that the config is
> > internal, so not
> > >     > >>>> sure
> > >     > >>>>>> how to document this in detail. We should not expose the
> > internal
> > >     > >>>> config
> > >     > >>>>>> in the docs.
> > >     > >>>>>>
> > >     > >>>>>> About upgrading: why do we need have two rolling bounces
> > and encode
> > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > >     > >>>>>>
> > >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> > static, I
> > >     > >>>> don't
> > >     > >>>>>> see any reason why both should not work together and
> single
> > rolling
> > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > consumer and
> > >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> > and the
> > >     > >>> broker
> > >     > >>>>>> registers the [member.name, member.id] in the cache. Why
> > would
> > >     > >> this
> > >     > >>>>>> interfere with all other consumer that use dynamic
> > assignment?
> > >     > >>>>>>
> > >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> > need to
> > >     > >> check
> > >     > >>>> if
> > >     > >>>>>> the mapping [member.name, member.id] contains the send `
> > member.id`
> > >     > >>> --
> > >     > >>>> I
> > >     > >>>>>> don't think this is necessary -- it seems to be sufficient
> > to check
> > >     > >>> the
> > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as
> be
> > do
> > >     > >> today
> > >     > >>> --
> > >     > >>>>>> thus, checking `member.id` does not require any change
> > IMHO.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> -Matthias
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > >     > >>>>>>> @James
> > >     > >>>>>>>
> > >     > >>>>>>> What you described is true: the transition from dynamic
> to
> > static
> > >     > >>>>>>> memberships are not thought through yet. But I do not
> > think it is
> > >     > >>> an
> > >     > >>>>>>> impossible problem: note that we indeed moved the offset
> > commit
> > >     > >>> from
> > >     > >>>> ZK
> > >     > >>>>>> to
> > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> > first to
> > >     > >>>>>>> double-commits on both zk and coordinator, and then do a
> > second
> > >     > >>> round
> > >     > >>>>> to
> > >     > >>>>>>> turn the zk off.
> > >     > >>>>>>>
> > >     > >>>>>>> So just to throw a wild idea here: also following a
> > >     > >>>> two-rolling-bounce
> > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> > "static"
> > >     > >>> while
> > >     > >>>>>> keep
> > >     > >>>>>>> the registry-id field empty still, in this case, the
> > coordinator
> > >     > >>>> still
> > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> while
> > >     > >>> allowing
> > >     > >>>>> the
> > >     > >>>>>>> protocol to be set to "static"; after the first rolling
> > bounce,
> > >     > >> the
> > >     > >>>>> group
> > >     > >>>>>>> protocol is already "static", then a second rolling
> bounce
> > is
> > >     > >>>> triggered
> > >     > >>>>>> and
> > >     > >>>>>>> this time we set the registry-id.
> > >     > >>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>> Guozhang
> > >     > >>>>>>>
> > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > >     > >> wushujames@gmail.com>
> > >     > >>>>>> wrote:
> > >     > >>>>>>>
> > >     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
> > >     > >>>>>>>>
> > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > >     > >>>
> > >     > >>>>> wrote:
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > fields:
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
> > >     > >>> protocols.
> > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > pre-defined
> > >     > >>> member
> > >     > >>>>> id.
> > >     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
> > >     > >>>>>>>>> "group-change-timeout" value.
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> > protocols
> > >     > >>> for
> > >     > >>>>> all
> > >     > >>>>>>>>> group members: we accept the protocol on the first
> joined
> > >     > >> member
> > >     > >>> of
> > >     > >>>>> the
> > >     > >>>>>>>>> group, and if later joining members indicate a
> different
> > >     > >>> membership
> > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > value was
> > >     > >>>>> different
> > >     > >>>>>>>> to
> > >     > >>>>>>>>> the first joined member, we reject it as well.
> > >     > >>>>>>>>
> > >     > >>>>>>>>
> > >     > >>>>>>>> What will happen if we have an already-deployed
> > application that
> > >     > >>>> wants
> > >     > >>>>>> to
> > >     > >>>>>>>> switch to using static membership? Let’s say there are
> 10
> > >     > >>> instances
> > >     > >>>> of
> > >     > >>>>>> it.
> > >     > >>>>>>>> As the instances go through a rolling restart, they will
> > switch
> > >     > >>> from
> > >     > >>>>>>>> dynamic membership (the default?) to static membership.
> > As each
> > >     > >>> one
> > >     > >>>>>> leaves
> > >     > >>>>>>>> the group and restarts, they will be rejected from the
> > group
> > >     > >>>> (because
> > >     > >>>>>> the
> > >     > >>>>>>>> group is currently using dynamic membership). The group
> > will
> > >     > >>> shrink
> > >     > >>>>> down
> > >     > >>>>>>>> until there is 1 node handling all the traffic. After
> > that one
> > >     > >>>>> restarts,
> > >     > >>>>>>>> the group will switch over to static membership.
> > >     > >>>>>>>>
> > >     > >>>>>>>> Is that right? That means that the transition plan from
> > dynamic
> > >     > >> to
> > >     > >>>>>> static
> > >     > >>>>>>>> membership isn’t very smooth.
> > >     > >>>>>>>>
> > >     > >>>>>>>> I’m not really sure what can be done in this case. This
> > reminds
> > >     > >> me
> > >     > >>>> of
> > >     > >>>>>> the
> > >     > >>>>>>>> transition plans that were discussed for moving from
> > >     > >>> zookeeper-based
> > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was
> > also
> > >     > >>> hard,
> > >     > >>>>> and
> > >     > >>>>>>>> ultimately we decided not to build that.
> > >     > >>>>>>>>
> > >     > >>>>>>>> -James
> > >     > >>>>>>>>
> > >     > >>>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>
> > >     > >>>>
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> --
> > >     > >>>> -- Guozhang
> > >     > >>>>
> > >     > >>>
> > >     > >>
> > >     > >>
> > >     > >>
> > >     > >> --
> > >     > >> -- Guozhang
> > >     > >>
> > >     >
> > >
> > >
> >
> >
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


--
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Thanks for the reply.

Please find the replies inline below :
For having a consumer config at runtime, I think it's not necessary to
address in this KIP because most companies run sidecar jobs through daemon
software like puppet. It should be easy to change the config through script
or UI without actual code change. We still want to leave flexibility for
user to define member name as they like.
---- This might be little different for companies that use configuration
management tools that does not allow the applications to define/change the
configs dynamically. For example, if we use something similar to spring to
pull in the configs for the KafkaConsumer and pass it to the constructor to
create the KafkaConsumer object, it will be hard to specify a unique value
to the "MEMBER_NAME" config unless someone deploying the app generates a
unique string for this config outside the deployment workflow and copies it
statically before starting up each consumer instance. Unless we can loosen
the criteria for uniqueness of this config value, for each consumer
instance in the consumer group, I am not sure of a better way of
addressing this. If we don't want to loosen the criteria, then providing a
dynamic way to pass this in at runtime, would put the onus of having the
same unique value each time a consumer is restarted, on to the application
that is running the consumer.

I just updated the kip about having both "registration timeout" and
"session timeout". The benefit of having two configs instead of one is to
reduce the mental burden for operation, for example user just needs to
unset "member name" to cast back to dynamic membership without worrying
about tuning the "session timeout" back to a smaller value.
--- That is a good point. I was thinking, if both the configs are
specified, it would be confusing for the end user without understanding the
internals of the consumer and its interaction with group coordinator, as
which takes precedence when and how it affects the consumer behavior. Just
my 2 cents.

Thanks,

Mayuresh

On Fri, Nov 9, 2018 at 8:27 PM Boyang Chen <bc...@outlook.com> wrote:

> Hey Mayuresh,
>
>
> thanks for the thoughtful questions! Let me try to answer your questions
> one by one.
>
>
> For having a consumer config at runtime, I think it's not necessary to
> address in this KIP because most companies run sidecar jobs through daemon
> software like puppet. It should be easy to change the config through script
> or UI without actual code change. We still want to leave flexibility for
> user to define member name as they like.
>
>
> I just updated the kip about having both "registration timeout" and
> "session timeout". The benefit of having two configs instead of one is to
> reduce the mental burden for operation, for example user just needs to
> unset "member name" to cast back to dynamic membership without worrying
> about tuning the "session timeout" back to a smaller value.
>
>
> For backup topic, I think it's a low-level detail which could be addressed
> in the implementation. I feel no preference of adding a new topic vs reuse
> consumer offsets topic. I will do more analysis and make a trade-off
> comparison. Nice catch!
>
>
> I hope the explanations make sense to you. I will keep polishing on the
> edge cases and details.
>
>
> Best,
>
> Boyang
>
> ________________________________
> From: Mayuresh Gharat <gh...@gmail.com>
> Sent: Saturday, November 10, 2018 10:25 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Thanks for the KIP and sorry for being late to the party. This KIP is
> really useful for us at Linkedin.
>
> I had a few questions :
>
> The idea of having static member name seems nice, but instead of a config,
> would it be possible for it to be passed in to the consumer at runtime?
> This is because an app might want to decide the config value at runtime
> using its host information for example, to generate the unique member name.
>
> Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
> wondering if we can reuse the session timeout here. This might help us to
> have one less config on the consumer.
>
> The KIP also talks about adding another internal topic "static_member_map".
> Would the semantics (GroupCoordinator broker, topic configs) be the same as
> __consumer_offsets topic?
>
> Thanks,
>
> Mayuresh
>
>
> On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:
>
> > I took a quick pass of the proposal. First I would say it's a very
> > brilliant initiative from Konstantine and Confluent folks. To draft up a
> > proposal like this needs deep understanding of the rebalance protocol! I
> > summarized some thoughts here.
> >
> >
> > Overall the motivations of the two proposals align on that:
> >
> >   1.  Both believe the invariant resource (belonging to the same process)
> > should be preserved across rebalance.
> >   2.  Transit failures (K8 thread death) shouldn't trigger resource
> > redistribution. I don't use rebalance here since part one of the
> > cooperative proposal could potentially introduce more rebalances but only
> > on must-move resources.
> >   3.  Scale up/down and rolling bounce are causing unnecessary resource
> > shuffling that need to be mitigated.
> >
> >
> > On motivation level, I think both approach could solve/mitigate the above
> > issues. They are just different in design philosophy, or I would say the
> > perspective difference between framework user and algorithm designer.
> >
> >
> > Two proposals have different focuses. KIP-345 is trying to place more
> > fine-grained control on the broker side to reduce the unnecessary
> > rebalances, while keeping the client logic intact. This is pretty
> intuitive
> > cause-effect for normal developers who are not very familiar with
> rebalance
> > protocol. As a developer working with Kafka Streams daily, I'd be happy
> to
> > see a simplified rebalance protocol and just focus on maintaining the
> > stream/consumer jobs. Too many rebalances raised my concern on the job
> > health. To be concise, static membership has the advantage of reducing
> > mental burden.
> >
> >
> > Cooperative proposal takes thoughtful approach on client side. We want to
> > have fine-grained control on the join/exit group behaviors and make the
> > current dynamic membership better to address above issues. I do feel our
> > idea crossed on the delayed rebalance when we scale up/down, which could
> > potentially reduce the state shuffling and decouple the behavior from
> > session timeout which is already overloaded.  In this sense, I believe
> both
> > approaches would serve well in making "reasonable rebalance" happen at
> the
> > "right timing".
> >
> >
> > However, based on my understanding, either 345 or cooperative rebalancing
> > is not solving the problem Mike has proposed: could we do a better job at
> > scaling up/down in ideal timing? My initial response was to introduce an
> > admin API which now I feel is sub-optimal, in that the goal of smooth
> > transition is to make sure the newly up hosts are actually "ready". For
> > example:
> >
> >
> > We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> > time we would like to scale up to 8 hosts, with the current improvements
> we
> > could reduce 4 potential rebalances to a single one. But the new hosts
> are
> > yet unknown to be "ready" if they need to reconstruct the local state. To
> > be actually ready, we need 4 standby tasks running on those empty hosts
> and
> > leader needs to wait for the signal of "replay/reconstruct complete" to
> > actually involve them into the main consumer group. Otherwise, rebalance
> > just kills our performance since we need to wait indefinite long for task
> > migration.
> >
> >
> > The scale down is also tricky such that we are not able to define a
> "true"
> > leave of a member. Rebalance immediately after "true" leaves are most
> > optimal comparing with human intervention. Does this make sense?
> >
> >
> > My intuition is that cooperative approach which was implemented on the
> > client side could better handle scaling cases than KIP 345, since it
> > involves a lot of algorithmic changes to define "replaying" stage, which
> I
> > feel would over-complicate broker logic if implemented on coordinator. If
> > we let 345 focus on reducing unnecessary rebalance, and let cooperative
> > approach focus on judging best timing of scale up/down, the two efforts
> > could be aligned. In long term, I feel the more complex improvement of
> > consumer protocol should happen on client side instead of server side
> which
> > is easier to test and has less global impact for the entire Kafka
> > production cluster.
> >
> >
> > Thanks again to Konstantine, Matthias and other folks in coming up with
> > this great client proposal. This is great complementation to KIP 345. In
> a
> > high level, we are not having any collision on the path and both
> proposals
> > are making sense here. Just need better sync to avoid duplicate effort :)
> >
> >
> > Best,
> >
> > Boyang
> >
> >
> > ________________________________
> > From: Boyang Chen <bc...@outlook.com>
> > Sent: Wednesday, November 7, 2018 1:57 PM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Thanks Matthias for bringing this awesome proposal up! I shall take a
> > deeper look and make a comparison between the two proposals.
> >
> >
> > Meanwhile for the scale down specifically for stateful streaming, we
> could
> > actually introduce a new status called "learner" where the newly up hosts
> > could try to catch up with the assigned task progress first before
> > triggering the rebalance, from which we don't see a sudden dip on the
> > progress. However, it is built on top of the success of KIP-345.
> >
> >
> > ________________________________
> > From: Matthias J. Sax <ma...@confluent.io>
> > Sent: Wednesday, November 7, 2018 7:02 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> >
> > Hey,
> >
> > there was quite a pause on this KIP discussion and in the mean time, a
> > new design for incremental cooporative rebalance was suggested:
> >
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
> Incremental Cooperative Rebalancing: Support and Policies ...<
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
> >
> cwiki.apache.org
> Rebalancing between distributed application processes in Apache Kafka was
> enhanced considerably when it was decoupled as logic from Kafka brokers and
> was moved as responsibility to the clients and specifically to Kafka
> Consumer. This pattern has been working robustly for quite a while now and
> has ...
>
>
>
> >
> >
> > We should make sure that the proposal and this KIP align to each other.
> > Thoughts?
> >
> >
> > -Matthias
> >
> > On 11/5/18 7:31 PM, Boyang Chen wrote:
> > > Hey Mike,
> > >
> > >
> > > thanks for the feedback, the two question are very thoughtful!
> > >
> > >
> > >> 1) I am a little confused about the distinction for the leader. If the
> > consumer node that was assigned leader does a bounce (goes down and
> quickly
> > comes up) to update application code, will a rebalance be triggered? I >
> do
> > not think a bounce of the leader should trigger a rebalance.
> > >
> > > For Q1 my intention was to minimize the change within one KIP, since
> the
> > leader rejoining case could be addressed separately.
> > >
> > >
> > >> 2) The timeout for shrink up makes a lot of sense and allows to
> > gracefully increase the number of nodes in the cluster. I think we need
> to
> > support graceful shrink down as well. If I set the registration timeout
> to
> > 5 minutes > to handle rolling restarts or intermittent failures without
> > shuffling state, I don't want to wait 5 minutes in order for the group to
> > rebalance if I am intentionally removing a node from the cluster. I am
> not
> > sure the best way to > do this. One idea I had was adding the ability
> for a
> > CLI or Admin API to force a rebalance of the group. This would allow for
> an
> > admin to trigger the rebalance manually without waiting the entire
> > registration timeout on > shrink down. What do you think?
> > >
> > > For 2) my understanding is that for scaling down case it is better to
> be
> > addressed by CLI tool than code logic, since only by human evaluation we
> > could decide whether it is a "right timing" -- the time when all the
> > scaling down consumers are offline -- to kick in rebalance. Unless we
> > introduce another term on coordinator which indicates the target consumer
> > group size, broker will find it hard to decide when to start rebalance.
> So
> > far I prefer to hold the implementation for that, but agree we could
> > discuss whether we want to introduce admin API in this KIP or a separate
> > one.
> > >
> > >
> > > Thanks again for the proposed ideas!
> > >
> > >
> > > Boyang
> > >
> > > ________________________________
> > > From: Mike Freyberger <mi...@xandr.com>
> > > Sent: Monday, November 5, 2018 6:13 AM
> > > To: dev@kafka.apache.org
> > > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> > specifying member id
> > >
> > > Boyang,
> > >
> > > Thanks for updating the KIP. It's shaping up well. Two things:
> > >
> > > 1) I am a little confused about the distinction for the leader. If the
> > consumer node that was assigned leader does a bounce (goes down and
> quickly
> > comes up) to update application code, will a rebalance be triggered? I do
> > not think a bounce of the leader should trigger a rebalance.
> > >
> > > 2) The timeout for shrink up makes a lot of sense and allows to
> > gracefully increase the number of nodes in the cluster. I think we need
> to
> > support graceful shrink down as well. If I set the registration timeout
> to
> > 5 minutes to handle rolling restarts or intermittent failures without
> > shuffling state, I don't want to wait 5 minutes in order for the group to
> > rebalance if I am intentionally removing a node from the cluster. I am
> not
> > sure the best way to do this. One idea I had was adding the ability for a
> > CLI or Admin API to force a rebalance of the group. This would allow for
> an
> > admin to trigger the rebalance manually without waiting the entire
> > registration timeout on shrink down. What do you think?
> > >
> > > Mike
> > >
> > > On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> > >
> > >     Btw, I updated KIP 345 based on my understanding. Feel free to take
> > another round of look:
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > > KIP-345: Introduce static membership protocol to reduce ...<
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > >
> > > cwiki.apache.org
> > > For stateful applications, one of the biggest performance bottleneck is
> > the state shuffling. In Kafka consumer, there is a concept called
> > "rebalance" which means that for given M partitions and N consumers in
> one
> > consumer group, Kafka will try to balance the load between consumers and
> > ideally have ...
> > >
> > >
> > >
> > >
> > >     KIP-345: Introduce static membership protocol to reduce ...<
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > >
> > >     cwiki.apache.org
> > >     For stateful applications, one of the biggest performance
> bottleneck
> > is the state shuffling. In Kafka consumer, there is a concept called
> > "rebalance" which means that for given M partitions and N consumers in
> one
> > consumer group, Kafka will try to balance the load between consumers and
> > ideally have ...
> > >
> > >
> > >
> > >
> > >
> > >     ________________________________
> > >     From: Boyang Chen <bc...@outlook.com>
> > >     Sent: Monday, October 29, 2018 12:34 PM
> > >     To: dev@kafka.apache.org
> > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by specifying member id
> > >
> > >     Thanks everyone for the input on this thread! (Sorry it's been a
> > while) I feel that we are very close to the final solution.
> > >
> > >
> > >     Hey Jason and Mike, I have two quick questions on the new features
> > here:
> > >
> > >       1.  so our proposal is that until we add a new static member into
> > the group (scale up), we will not trigger rebalance until the
> "registration
> > timeout"( the member has been offline for too long)? How about leader's
> > rejoin request, I think we should still trigger rebalance when that
> > happens, since the consumer group may have new topics to consume?
> > >       2.  I'm not very clear on the scale up scenario in static
> > membership here. Should we fallback to dynamic membership while
> > adding/removing hosts (by setting member.name = null), or we still want
> > to add instances with `member.name` so that we eventually expand/shrink
> > the static membership? I personally feel the easier solution is to spin
> up
> > new members and wait until either the same "registration timeout" or a
> > "scale up timeout" before starting the rebalance. What do you think?
> > >
> > >     Meanwhile I will go ahead to make changes to the KIP with our newly
> > discussed items and details. Really excited to see the design has become
> > more solid.
> > >
> > >     Best,
> > >     Boyang
> > >
> > >     ________________________________
> > >     From: Jason Gustafson <ja...@confluent.io>
> > >     Sent: Saturday, August 25, 2018 6:04 AM
> > >     To: dev
> > >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> > by specifying member id
> > >
> > >     Hey Mike,
> > >
> > >     Yeah, that's a good point. A long "registration timeout" may not be
> > a great
> > >     idea. Perhaps in practice you'd set it long enough to be able to
> > detect a
> > >     failure and provision a new instance. Maybe on the order of 10
> > minutes is
> > >     more reasonable.
> > >
> > >     In any case, it's probably a good idea to have an administrative
> way
> > to
> > >     force deregistration. One option is to extend the DeleteGroups API
> > with a
> > >     list of members names.
> > >
> > >     -Jason
> > >
> > >
> > >
> > >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> > mfreyberger@appnexus.com>
> > >     wrote:
> > >
> > >     > Jason,
> > >     >
> > >     > Regarding step 4 in your proposal which suggests beginning a long
> > timer
> > >     > (30 minutes) when a static member leaves the group, would there
> > also be the
> > >     > ability for an admin to force a static membership expiration?
> > >     >
> > >     > I’m thinking that during particular types of outages or upgrades
> > users
> > >     > would want forcefully remove a static member from the group.
> > >     >
> > >     > So the user would shut the consumer down normally, which wouldn’t
> > trigger
> > >     > a rebalance. Then the user could use an admin CLI tool to force
> > remove that
> > >     > consumer from the group, so the TopicPartitions that were
> > previously owned
> > >     > by that consumer can be released.
> > >     >
> > >     > At a high level, we need consumer groups to gracefully handle
> > intermittent
> > >     > failures and permanent failures. Currently, the consumer group
> > protocol
> > >     > handles permanent failures well, but does not handle intermittent
> > failures
> > >     > well (it creates unnecessary rebalances). I want to make sure the
> > overall
> > >     > solution here handles both intermittent failures and permanent
> > failures,
> > >     > rather than sacrificing support for permanent failures in order
> to
> > provide
> > >     > support for intermittent failures.
> > >     >
> > >     > Mike
> > >     >
> > >     > Sent from my iPhone
> > >     >
> > >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <
> jason@confluent.io>
> > wrote:
> > >     > >
> > >     > > Hey Guozhang,
> > >     > >
> > >     > > Responses below:
> > >     > >
> > >     > > Originally I was trying to kill more birds with one stone with
> > KIP-345,
> > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> shutting
> > down a
> > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > email), and
> > >     > >> hence proposing to have a pure static-membership protocol. But
> > thinking
> > >     > >> twice about it I now feel it may be too ambitious and worth
> > fixing in
> > >     > >> another KIP.
> > >     > >
> > >     > >
> > >     > > I was considering an extension to support pre-initialization of
> > the
> > >     > static
> > >     > > members of the group, but I agree we should probably leave this
> > problem
> > >     > for
> > >     > > future work.
> > >     > >
> > >     > > 1. How this longish static member expiration timeout defined?
> Is
> > it via a
> > >     > >> broker, hence global config, or via a client config which can
> be
> > >     > >> communicated to broker via JoinGroupRequest?
> > >     > >
> > >     > >
> > >     > > I am not too sure. I tend to lean toward server-side configs
> > because they
> > >     > > are easier to evolve. If we have to add something to the
> > protocol, then
> > >     > > we'll be stuck with it forever.
> > >     > >
> > >     > > 2. Assuming that for static members, LEAVE_GROUP request will
> not
> > >     > trigger a
> > >     > >> rebalance immediately either, similar to session timeout, but
> > only the
> > >     > >> longer member expiration timeout, can we remove the internal "
> > >     > >> internal.leave.group.on.close" config, which is a quick
> > walk-around
> > >     > then?
> > >     > >
> > >     > >
> > >     > > Yeah, I hope we can ultimately get rid of it, but we may need
> it
> > for
> > >     > > compatibility with older brokers. A related question is what
> > should be
> > >     > the
> > >     > > behavior of the consumer if `member.name` is provided but the
> > broker
> > >     > does
> > >     > > not support it? We could either fail or silently downgrade to
> > dynamic
> > >     > > membership.
> > >     > >
> > >     > > -Jason
> > >     > >
> > >     > >
> > >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> > wangguoz@gmail.com>
> > >     > wrote:
> > >     > >>
> > >     > >> Hey Jason,
> > >     > >>
> > >     > >> I like your idea to simplify the upgrade protocol to allow
> > co-exist of
> > >     > >> static and dynamic members. Admittedly it may make the
> > coordinator-side
> > >     > >> logic a bit more complex, but I think it worth doing it.
> > >     > >>
> > >     > >> Originally I was trying to kill more birds with one stone with
> > KIP-345,
> > >     > >> e.g. to fix the multi-rebalance issue on starting up /
> shutting
> > down a
> > >     > >> multi-instance client (mentioned as case 1)/2) in my early
> > email), and
> > >     > >> hence proposing to have a pure static-membership protocol. But
> > thinking
> > >     > >> twice about it I now feel it may be too ambitious and worth
> > fixing in
> > >     > >> another KIP. With that, I think what you've proposed here is a
> > good way
> > >     > to
> > >     > >> go for KIP-345 itself.
> > >     > >>
> > >     > >> Note there are a few details in your proposal we'd still need
> > to figure
> > >     > >> out:
> > >     > >>
> > >     > >> 1. How this longish static member expiration timeout defined?
> > Is it via
> > >     > a
> > >     > >> broker, hence global config, or via a client config which can
> be
> > >     > >> communicated to broker via JoinGroupRequest?
> > >     > >>
> > >     > >> 2. Assuming that for static members, LEAVE_GROUP request will
> > not
> > >     > trigger a
> > >     > >> rebalance immediately either, similar to session timeout, but
> > only the
> > >     > >> longer member expiration timeout, can we remove the internal "
> > >     > >> internal.leave.group.on.close" config, which is a quick
> > walk-around
> > >     > then?
> > >     > >>
> > >     > >>
> > >     > >>
> > >     > >> Guozhang
> > >     > >>
> > >     > >>
> > >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> > jason@confluent.io>
> > >     > >> wrote:
> > >     > >>
> > >     > >>> Hey All,
> > >     > >>>
> > >     > >>> Nice to see some solid progress on this. It sounds like one
> of
> > the
> > >     > >>> complications is allowing static and dynamic registration to
> > coexist.
> > >     > I'm
> > >     > >>> wondering if we can do something like the following:
> > >     > >>>
> > >     > >>> 1. Statically registered members (those joining the group
> with
> > a
> > >     > >> non-null `
> > >     > >>> member.name`) maintain a session with the coordinator just
> > like
> > >     > dynamic
> > >     > >>> members.
> > >     > >>> 2. If a session is active for a static member when a
> rebalance
> > begins,
> > >     > >> then
> > >     > >>> basically we'll keep the current behavior. The rebalance will
> > await the
> > >     > >>> static member joining the group.
> > >     > >>> 3. If a static member does not have an active session, then
> the
> > >     > >> coordinator
> > >     > >>> will not wait for it to join, but will still include it in
> the
> > >     > rebalance.
> > >     > >>> The coordinator will forward the cached subscription
> > information to the
> > >     > >>> leader and will cache the assignment after the rebalance
> > completes.
> > >     > (Note
> > >     > >>> that we still have the generationId to fence offset commits
> > from a
> > >     > static
> > >     > >>> zombie if the assignment changes.)
> > >     > >>> 4. When a static member leaves the group or has its session
> > expire, no
> > >     > >>> rebalance is triggered. Instead, we can begin a timer to
> > expire the
> > >     > >> static
> > >     > >>> registration. This would be a longish timeout (like 30
> minutes
> > say).
> > >     > >>>
> > >     > >>> So basically static members participate in all rebalances
> > regardless
> > >     > >>> whether they have an active session. In a given rebalance,
> > some of the
> > >     > >>> members may be static and some dynamic. The group leader can
> > >     > >> differentiate
> > >     > >>> the two based on the presence of the `member.name` (we have
> > to add
> > >     > this
> > >     > >> to
> > >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> > leaders
> > >     > >>> preferentially from the active members that support the
> latest
> > >     > JoinGroup
> > >     > >>> protocol and are using static membership. If we have to
> choose
> > a leader
> > >     > >>> with an old version, however, it would see all members in the
> > group
> > >     > >> (static
> > >     > >>> or dynamic) as dynamic members and perform the assignment as
> > usual.
> > >     > >>>
> > >     > >>> Would that work?
> > >     > >>>
> > >     > >>> -Jason
> > >     > >>>
> > >     > >>>
> > >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> > wangguoz@gmail.com>
> > >     > >> wrote:
> > >     > >>>
> > >     > >>>> Hello Boyang,
> > >     > >>>>
> > >     > >>>> Thanks for the updated proposal, a few questions:
> > >     > >>>>
> > >     > >>>> 1. Where will "change-group-timeout" be communicated to the
> > broker?
> > >     > >> Will
> > >     > >>>> that be a new field in the JoinGroupRequest, or are we going
> > to
> > >     > >>> piggy-back
> > >     > >>>> on the existing session-timeout field (assuming that the
> > original
> > >     > value
> > >     > >>>> will not be used anywhere in the static membership any
> more)?
> > >     > >>>>
> > >     > >>>> 2. "However, if the consumer takes longer than session
> > timeout to
> > >     > >> return,
> > >     > >>>> we shall still trigger rebalance but it could still try to
> > catch
> > >     > >>>> `change-group-timeout`.": what does this mean? I thought
> your
> > proposal
> > >     > >> is
> > >     > >>>> that for static memberships, the broker will NOT trigger
> > rebalance
> > >     > even
> > >     > >>>> after session-timeout has been detected, but only that after
> > >     > >>>> change-group-timeout
> > >     > >>>> which is supposed to be longer than session-timeout to be
> > defined?
> > >     > >>>>
> > >     > >>>> 3. "A join group request with member.name set will be
> > treated as
> > >     > >>>> `static-membership` strategy", in this case, how would the
> > switch from
> > >     > >>>> dynamic to static happen, since whoever changed the
> > member.name to
> > >     > >>>> not-null
> > >     > >>>> will be rejected, right?
> > >     > >>>>
> > >     > >>>> 4. "just erase the cached mapping, and wait for session
> > timeout to
> > >     > >>> trigger
> > >     > >>>> rebalance should be sufficient." this is also a bit unclear
> > to me: who
> > >     > >>> will
> > >     > >>>> erase the cached mapping? Since it is on the broker-side I
> > assume that
> > >     > >>>> broker has to do it. Are you suggesting to use a new request
> > for it?
> > >     > >>>>
> > >     > >>>> 5. "Halfway switch": following 3) above, if your proposal is
> > basically
> > >     > >> to
> > >     > >>>> let "first join-request wins", and the strategy will stay as
> > is until
> > >     > >> all
> > >     > >>>> members are gone, then this will also not happen since
> > whoever used
> > >     > >>>> different strategy as the first guy who sends join-group
> > request will
> > >     > >> be
> > >     > >>>> rejected right?
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> Guozhang
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> > john@confluent.io>
> > >     > >> wrote:
> > >     > >>>>
> > >     > >>>>> This sounds good to me!
> > >     > >>>>>
> > >     > >>>>> Thanks for the time you've spent on it,
> > >     > >>>>> -John
> > >     > >>>>>
> > >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> > bchen11@outlook.com>
> > >     > >>>> wrote:
> > >     > >>>>>
> > >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently
> and
> > >     > >> haven't
> > >     > >>>> got
> > >     > >>>>>> time to update this thread. To summarize what we come up
> so
> > far,
> > >     > >> here
> > >     > >>>> is
> > >     > >>>>> a
> > >     > >>>>>> draft updated plan:
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Introduce a new config called `member.name` which is
> > supposed to
> > >     > >> be
> > >     > >>>>>> provided uniquely by the consumer client. The broker will
> > maintain
> > >     > >> a
> > >     > >>>>> cache
> > >     > >>>>>> with [key:member.name, value:member.id]. A join group
> > request with
> > >     > >>>>>> member.name set will be treated as `static-membership`
> > strategy,
> > >     > >> and
> > >     > >>>>> will
> > >     > >>>>>> reject any join group request without member.name. So
> this
> > >     > >>>> coordination
> > >     > >>>>>> change will be differentiated from the
> `dynamic-membership`
> > >     > >> protocol
> > >     > >>> we
> > >     > >>>>>> currently have.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When handling static join group request:
> > >     > >>>>>>
> > >     > >>>>>>  1.   The broker will check the membership to see whether
> > this is
> > >     > >> a
> > >     > >>>> new
> > >     > >>>>>> member. If new, broker allocate a unique member id, cache
> > the
> > >     > >> mapping
> > >     > >>>> and
> > >     > >>>>>> move to rebalance stage.
> > >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> > will not
> > >     > >>>> change
> > >     > >>>>>> group state, and return its cached member.id and current
> > >     > >> assignment.
> > >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> > >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with
> > pair
> > >     > >>> member
> > >     > >>>>>> name and id, I think for join group request it is ok to
> > leave
> > >     > >> member
> > >     > >>> id
> > >     > >>>>>> blank as member name is the unique identifier. In commit
> > offset
> > >     > >>> request
> > >     > >>>>> we
> > >     > >>>>>> *must* have both.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When handling commit offset request, if enabled with
> static
> > >     > >>> membership,
> > >     > >>>>>> each time the commit request must have both member.name
> and
> > >     > >>> member.id
> > >     > >>>> to
> > >     > >>>>>> be identified as a `certificated member`. If not, this
> > means there
> > >     > >>> are
> > >     > >>>>>> duplicate consumer members with same member name and the
> > request
> > >     > >> will
> > >     > >>>> be
> > >     > >>>>>> rejected to guarantee consumption uniqueness.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> When rolling restart/shutting down gracefully, the client
> > will
> > >     > >> send a
> > >     > >>>>>> leave group request (static membership mode). In static
> > membership,
> > >     > >>> we
> > >     > >>>>> will
> > >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> > provided by
> > >     > >>>>> leader.
> > >     > >>>>>> So we will wait for all the members to rejoin the group
> and
> > do
> > >     > >>> exactly
> > >     > >>>>> one
> > >     > >>>>>> rebalance since all members are expected to rejoin within
> > timeout.
> > >     > >> If
> > >     > >>>>>> consumer crashes, the join group request from the
> restarted
> > >     > >> consumer
> > >     > >>>> will
> > >     > >>>>>> be recognized as an existing member and be handled as
> above
> > >     > >> condition
> > >     > >>>> 1;
> > >     > >>>>>> However, if the consumer takes longer than session timeout
> > to
> > >     > >> return,
> > >     > >>>> we
> > >     > >>>>>> shall still trigger rebalance but it could still try to
> > catch
> > >     > >>>>>> `change-group-timeout`. If it failed to catch second
> > timeout, its
> > >     > >>>> cached
> > >     > >>>>>> state on broker will be garbage collected and trigger a
> new
> > >     > >> rebalance
> > >     > >>>>> when
> > >     > >>>>>> it finally joins.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> And consider the switch between dynamic to static
> > membership.
> > >     > >>>>>>
> > >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
> > >     > >> membership
> > >     > >>>> to
> > >     > >>>>>> static and wait for all the current members to restart,
> > since their
> > >     > >>>>>> membership is still dynamic. Here our assumption is that
> the
> > >     > >> restart
> > >     > >>>>>> process shouldn't take a long time, as long restart is
> > breaking the
> > >     > >>>>>> `rebalance timeout` in whatever membership protocol we are
> > using.
> > >     > >>>> Before
> > >     > >>>>>> restart, all dynamic member join requests will be
> rejected.
> > >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade
> which
> > should
> > >     > >>> be
> > >     > >>>>>> smooth: just erase the cached mapping, and wait for
> session
> > timeout
> > >     > >>> to
> > >     > >>>>>> trigger rebalance should be sufficient. (Fallback to
> current
> > >     > >>> behavior)
> > >     > >>>>>>  3.  Halfway switch: a corner case is like some clients
> keep
> > >     > >> dynamic
> > >     > >>>>>> membership while some keep static membership. This will
> > cause the
> > >     > >>> group
> > >     > >>>>>> rebalance forever without progress because dynamic/static
> > states
> > >     > >> are
> > >     > >>>>>> bouncing each other. This could guarantee that we will not
> > make the
> > >     > >>>>>> consumer group work in a wrong state by having half static
> > and half
> > >     > >>>>> dynamic.
> > >     > >>>>>>
> > >     > >>>>>> To guarantee correctness, we will also push the member
> > name/id pair
> > >     > >>> to
> > >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> > upgrade the
> > >     > >> API
> > >     > >>>>>> version, these details will be further discussed back in
> > the KIP.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Are there any concern for this high level proposal? Just
> > want to
> > >     > >>>>> reiterate
> > >     > >>>>>> on the core idea of the KIP: "If the broker recognize this
> > consumer
> > >     > >>> as
> > >     > >>>> an
> > >     > >>>>>> existing member, it shouldn't trigger rebalance".
> > >     > >>>>>>
> > >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is
> > much
> > >     > >> more
> > >     > >>>>>> robust than previous one!
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> Best,
> > >     > >>>>>>
> > >     > >>>>>> Boyang
> > >     > >>>>>>
> > >     > >>>>>> ________________________________
> > >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> > >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> > >     > >>>>>> To: dev@kafka.apache.org
> > >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> > rebalances
> > >     > >>> by
> > >     > >>>>>> specifying member id
> > >     > >>>>>>
> > >     > >>>>>> Hi,
> > >     > >>>>>>
> > >     > >>>>>> thanks for the detailed discussion. I learned a lot about
> > internals
> > >     > >>>> again
> > >     > >>>>>> :)
> > >     > >>>>>>
> > >     > >>>>>> I like the idea or a user config `member.name` and to
> keep
> > `
> > >     > >>> member.id`
> > >     > >>>>>> internal. Also agree with Guozhang, that reusing `
> client.id`
> > might
> > >     > >>> not
> > >     > >>>>>> be a good idea.
> > >     > >>>>>>
> > >     > >>>>>> To clarify the algorithm, each time we generate a new `
> > member.id`,
> > >     > >>> we
> > >     > >>>>>> also need to update the "group membership" information
> (ie,
> > mapping
> > >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id`
> > replaces
> > >     > >>> the
> > >     > >>>>>> old entry in the cache.
> > >     > >>>>>>
> > >     > >>>>>> I also think, we need to preserve the `member.name ->
> > member.id`
> > >     > >>>> mapping
> > >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> > this IMHO.
> > >     > >>>>>>
> > >     > >>>>>> For changing the default value of config
> > `leave.group.on.close`. I
> > >     > >>>> agree
> > >     > >>>>>> with John, that we should not change the default config,
> > because it
> > >     > >>>>>> would impact all consumer groups with dynamic assignment.
> > However,
> > >     > >> I
> > >     > >>>>>> think we can document, that if static assignment is used
> > (ie,
> > >     > >>>>>> `member.name` is configured) we never send a
> > LeaveGroupRequest
> > >     > >>>>>> regardless of the config. Note, that the config is
> > internal, so not
> > >     > >>>> sure
> > >     > >>>>>> how to document this in detail. We should not expose the
> > internal
> > >     > >>>> config
> > >     > >>>>>> in the docs.
> > >     > >>>>>>
> > >     > >>>>>> About upgrading: why do we need have two rolling bounces
> > and encode
> > >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> > >     > >>>>>>
> > >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> > static, I
> > >     > >>>> don't
> > >     > >>>>>> see any reason why both should not work together and
> single
> > rolling
> > >     > >>>>>> bounce would not be sufficient? If we bounce the first
> > consumer and
> > >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> > and the
> > >     > >>> broker
> > >     > >>>>>> registers the [member.name, member.id] in the cache. Why
> > would
> > >     > >> this
> > >     > >>>>>> interfere with all other consumer that use dynamic
> > assignment?
> > >     > >>>>>>
> > >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> > need to
> > >     > >> check
> > >     > >>>> if
> > >     > >>>>>> the mapping [member.name, member.id] contains the send `
> > member.id`
> > >     > >>> --
> > >     > >>>> I
> > >     > >>>>>> don't think this is necessary -- it seems to be sufficient
> > to check
> > >     > >>> the
> > >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as
> be
> > do
> > >     > >> today
> > >     > >>> --
> > >     > >>>>>> thus, checking `member.id` does not require any change
> > IMHO.
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>> -Matthias
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> > >     > >>>>>>> @James
> > >     > >>>>>>>
> > >     > >>>>>>> What you described is true: the transition from dynamic
> to
> > static
> > >     > >>>>>>> memberships are not thought through yet. But I do not
> > think it is
> > >     > >>> an
> > >     > >>>>>>> impossible problem: note that we indeed moved the offset
> > commit
> > >     > >>> from
> > >     > >>>> ZK
> > >     > >>>>>> to
> > >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> > first to
> > >     > >>>>>>> double-commits on both zk and coordinator, and then do a
> > second
> > >     > >>> round
> > >     > >>>>> to
> > >     > >>>>>>> turn the zk off.
> > >     > >>>>>>>
> > >     > >>>>>>> So just to throw a wild idea here: also following a
> > >     > >>>> two-rolling-bounce
> > >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> > "static"
> > >     > >>> while
> > >     > >>>>>> keep
> > >     > >>>>>>> the registry-id field empty still, in this case, the
> > coordinator
> > >     > >>>> still
> > >     > >>>>>>> follows the logic of "dynamic", accepting the request
> while
> > >     > >>> allowing
> > >     > >>>>> the
> > >     > >>>>>>> protocol to be set to "static"; after the first rolling
> > bounce,
> > >     > >> the
> > >     > >>>>> group
> > >     > >>>>>>> protocol is already "static", then a second rolling
> bounce
> > is
> > >     > >>>> triggered
> > >     > >>>>>> and
> > >     > >>>>>>> this time we set the registry-id.
> > >     > >>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>> Guozhang
> > >     > >>>>>>>
> > >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> > >     > >> wushujames@gmail.com>
> > >     > >>>>>> wrote:
> > >     > >>>>>>>
> > >     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
> > >     > >>>>>>>>
> > >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> > wangguoz@gmail.com
> > >     > >>>
> > >     > >>>>> wrote:
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> > fields:
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
> > >     > >>> protocols.
> > >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> > pre-defined
> > >     > >>> member
> > >     > >>>>> id.
> > >     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
> > >     > >>>>>>>>> "group-change-timeout" value.
> > >     > >>>>>>>>>
> > >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> > protocols
> > >     > >>> for
> > >     > >>>>> all
> > >     > >>>>>>>>> group members: we accept the protocol on the first
> joined
> > >     > >> member
> > >     > >>> of
> > >     > >>>>> the
> > >     > >>>>>>>>> group, and if later joining members indicate a
> different
> > >     > >>> membership
> > >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> > value was
> > >     > >>>>> different
> > >     > >>>>>>>> to
> > >     > >>>>>>>>> the first joined member, we reject it as well.
> > >     > >>>>>>>>
> > >     > >>>>>>>>
> > >     > >>>>>>>> What will happen if we have an already-deployed
> > application that
> > >     > >>>> wants
> > >     > >>>>>> to
> > >     > >>>>>>>> switch to using static membership? Let’s say there are
> 10
> > >     > >>> instances
> > >     > >>>> of
> > >     > >>>>>> it.
> > >     > >>>>>>>> As the instances go through a rolling restart, they will
> > switch
> > >     > >>> from
> > >     > >>>>>>>> dynamic membership (the default?) to static membership.
> > As each
> > >     > >>> one
> > >     > >>>>>> leaves
> > >     > >>>>>>>> the group and restarts, they will be rejected from the
> > group
> > >     > >>>> (because
> > >     > >>>>>> the
> > >     > >>>>>>>> group is currently using dynamic membership). The group
> > will
> > >     > >>> shrink
> > >     > >>>>> down
> > >     > >>>>>>>> until there is 1 node handling all the traffic. After
> > that one
> > >     > >>>>> restarts,
> > >     > >>>>>>>> the group will switch over to static membership.
> > >     > >>>>>>>>
> > >     > >>>>>>>> Is that right? That means that the transition plan from
> > dynamic
> > >     > >> to
> > >     > >>>>>> static
> > >     > >>>>>>>> membership isn’t very smooth.
> > >     > >>>>>>>>
> > >     > >>>>>>>> I’m not really sure what can be done in this case. This
> > reminds
> > >     > >> me
> > >     > >>>> of
> > >     > >>>>>> the
> > >     > >>>>>>>> transition plans that were discussed for moving from
> > >     > >>> zookeeper-based
> > >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was
> > also
> > >     > >>> hard,
> > >     > >>>>> and
> > >     > >>>>>>>> ultimately we decided not to build that.
> > >     > >>>>>>>>
> > >     > >>>>>>>> -James
> > >     > >>>>>>>>
> > >     > >>>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>>
> > >     > >>>>>>
> > >     > >>>>>>
> > >     > >>>>>
> > >     > >>>>
> > >     > >>>>
> > >     > >>>>
> > >     > >>>> --
> > >     > >>>> -- Guozhang
> > >     > >>>>
> > >     > >>>
> > >     > >>
> > >     > >>
> > >     > >>
> > >     > >> --
> > >     > >> -- Guozhang
> > >     > >>
> > >     >
> > >
> > >
> >
> >
>
> --
> -Regards,
> Mayuresh R. Gharat
> (862) 250-7125
>


-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Hey Mayuresh,


thanks for the thoughtful questions! Let me try to answer your questions one by one.


For having a consumer config at runtime, I think it's not necessary to address in this KIP because most companies run sidecar jobs through daemon software like puppet. It should be easy to change the config through script or UI without actual code change. We still want to leave flexibility for user to define member name as they like.


I just updated the kip about having both "registration timeout" and "session timeout". The benefit of having two configs instead of one is to reduce the mental burden for operation, for example user just needs to unset "member name" to cast back to dynamic membership without worrying about tuning the "session timeout" back to a smaller value.


For backup topic, I think it's a low-level detail which could be addressed in the implementation. I feel no preference of adding a new topic vs reuse consumer offsets topic. I will do more analysis and make a trade-off comparison. Nice catch!


I hope the explanations make sense to you. I will keep polishing on the edge cases and details.


Best,

Boyang

________________________________
From: Mayuresh Gharat <gh...@gmail.com>
Sent: Saturday, November 10, 2018 10:25 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hi Boyang,

Thanks for the KIP and sorry for being late to the party. This KIP is
really useful for us at Linkedin.

I had a few questions :

The idea of having static member name seems nice, but instead of a config,
would it be possible for it to be passed in to the consumer at runtime?
This is because an app might want to decide the config value at runtime
using its host information for example, to generate the unique member name.

Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
wondering if we can reuse the session timeout here. This might help us to
have one less config on the consumer.

The KIP also talks about adding another internal topic "static_member_map".
Would the semantics (GroupCoordinator broker, topic configs) be the same as
__consumer_offsets topic?

Thanks,

Mayuresh


On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:

> I took a quick pass of the proposal. First I would say it's a very
> brilliant initiative from Konstantine and Confluent folks. To draft up a
> proposal like this needs deep understanding of the rebalance protocol! I
> summarized some thoughts here.
>
>
> Overall the motivations of the two proposals align on that:
>
>   1.  Both believe the invariant resource (belonging to the same process)
> should be preserved across rebalance.
>   2.  Transit failures (K8 thread death) shouldn't trigger resource
> redistribution. I don't use rebalance here since part one of the
> cooperative proposal could potentially introduce more rebalances but only
> on must-move resources.
>   3.  Scale up/down and rolling bounce are causing unnecessary resource
> shuffling that need to be mitigated.
>
>
> On motivation level, I think both approach could solve/mitigate the above
> issues. They are just different in design philosophy, or I would say the
> perspective difference between framework user and algorithm designer.
>
>
> Two proposals have different focuses. KIP-345 is trying to place more
> fine-grained control on the broker side to reduce the unnecessary
> rebalances, while keeping the client logic intact. This is pretty intuitive
> cause-effect for normal developers who are not very familiar with rebalance
> protocol. As a developer working with Kafka Streams daily, I'd be happy to
> see a simplified rebalance protocol and just focus on maintaining the
> stream/consumer jobs. Too many rebalances raised my concern on the job
> health. To be concise, static membership has the advantage of reducing
> mental burden.
>
>
> Cooperative proposal takes thoughtful approach on client side. We want to
> have fine-grained control on the join/exit group behaviors and make the
> current dynamic membership better to address above issues. I do feel our
> idea crossed on the delayed rebalance when we scale up/down, which could
> potentially reduce the state shuffling and decouple the behavior from
> session timeout which is already overloaded.  In this sense, I believe both
> approaches would serve well in making "reasonable rebalance" happen at the
> "right timing".
>
>
> However, based on my understanding, either 345 or cooperative rebalancing
> is not solving the problem Mike has proposed: could we do a better job at
> scaling up/down in ideal timing? My initial response was to introduce an
> admin API which now I feel is sub-optimal, in that the goal of smooth
> transition is to make sure the newly up hosts are actually "ready". For
> example:
>
>
> We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> time we would like to scale up to 8 hosts, with the current improvements we
> could reduce 4 potential rebalances to a single one. But the new hosts are
> yet unknown to be "ready" if they need to reconstruct the local state. To
> be actually ready, we need 4 standby tasks running on those empty hosts and
> leader needs to wait for the signal of "replay/reconstruct complete" to
> actually involve them into the main consumer group. Otherwise, rebalance
> just kills our performance since we need to wait indefinite long for task
> migration.
>
>
> The scale down is also tricky such that we are not able to define a "true"
> leave of a member. Rebalance immediately after "true" leaves are most
> optimal comparing with human intervention. Does this make sense?
>
>
> My intuition is that cooperative approach which was implemented on the
> client side could better handle scaling cases than KIP 345, since it
> involves a lot of algorithmic changes to define "replaying" stage, which I
> feel would over-complicate broker logic if implemented on coordinator. If
> we let 345 focus on reducing unnecessary rebalance, and let cooperative
> approach focus on judging best timing of scale up/down, the two efforts
> could be aligned. In long term, I feel the more complex improvement of
> consumer protocol should happen on client side instead of server side which
> is easier to test and has less global impact for the entire Kafka
> production cluster.
>
>
> Thanks again to Konstantine, Matthias and other folks in coming up with
> this great client proposal. This is great complementation to KIP 345. In a
> high level, we are not having any collision on the path and both proposals
> are making sense here. Just need better sync to avoid duplicate effort :)
>
>
> Best,
>
> Boyang
>
>
> ________________________________
> From: Boyang Chen <bc...@outlook.com>
> Sent: Wednesday, November 7, 2018 1:57 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Thanks Matthias for bringing this awesome proposal up! I shall take a
> deeper look and make a comparison between the two proposals.
>
>
> Meanwhile for the scale down specifically for stateful streaming, we could
> actually introduce a new status called "learner" where the newly up hosts
> could try to catch up with the assigned task progress first before
> triggering the rebalance, from which we don't see a sudden dip on the
> progress. However, it is built on top of the success of KIP-345.
>
>
> ________________________________
> From: Matthias J. Sax <ma...@confluent.io>
> Sent: Wednesday, November 7, 2018 7:02 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey,
>
> there was quite a pause on this KIP discussion and in the mean time, a
> new design for incremental cooporative rebalance was suggested:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
Incremental Cooperative Rebalancing: Support and Policies ...<https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies>
cwiki.apache.org
Rebalancing between distributed application processes in Apache Kafka was enhanced considerably when it was decoupled as logic from Kafka brokers and was moved as responsibility to the clients and specifically to Kafka Consumer. This pattern has been working robustly for quite a while now and has ...



>
>
> We should make sure that the proposal and this KIP align to each other.
> Thoughts?
>
>
> -Matthias
>
> On 11/5/18 7:31 PM, Boyang Chen wrote:
> > Hey Mike,
> >
> >
> > thanks for the feedback, the two question are very thoughtful!
> >
> >
> >> 1) I am a little confused about the distinction for the leader. If the
> consumer node that was assigned leader does a bounce (goes down and quickly
> comes up) to update application code, will a rebalance be triggered? I > do
> not think a bounce of the leader should trigger a rebalance.
> >
> > For Q1 my intention was to minimize the change within one KIP, since the
> leader rejoining case could be addressed separately.
> >
> >
> >> 2) The timeout for shrink up makes a lot of sense and allows to
> gracefully increase the number of nodes in the cluster. I think we need to
> support graceful shrink down as well. If I set the registration timeout to
> 5 minutes > to handle rolling restarts or intermittent failures without
> shuffling state, I don't want to wait 5 minutes in order for the group to
> rebalance if I am intentionally removing a node from the cluster. I am not
> sure the best way to > do this. One idea I had was adding the ability for a
> CLI or Admin API to force a rebalance of the group. This would allow for an
> admin to trigger the rebalance manually without waiting the entire
> registration timeout on > shrink down. What do you think?
> >
> > For 2) my understanding is that for scaling down case it is better to be
> addressed by CLI tool than code logic, since only by human evaluation we
> could decide whether it is a "right timing" -- the time when all the
> scaling down consumers are offline -- to kick in rebalance. Unless we
> introduce another term on coordinator which indicates the target consumer
> group size, broker will find it hard to decide when to start rebalance. So
> far I prefer to hold the implementation for that, but agree we could
> discuss whether we want to introduce admin API in this KIP or a separate
> one.
> >
> >
> > Thanks again for the proposed ideas!
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Mike Freyberger <mi...@xandr.com>
> > Sent: Monday, November 5, 2018 6:13 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
> >
> > Boyang,
> >
> > Thanks for updating the KIP. It's shaping up well. Two things:
> >
> > 1) I am a little confused about the distinction for the leader. If the
> consumer node that was assigned leader does a bounce (goes down and quickly
> comes up) to update application code, will a rebalance be triggered? I do
> not think a bounce of the leader should trigger a rebalance.
> >
> > 2) The timeout for shrink up makes a lot of sense and allows to
> gracefully increase the number of nodes in the cluster. I think we need to
> support graceful shrink down as well. If I set the registration timeout to
> 5 minutes to handle rolling restarts or intermittent failures without
> shuffling state, I don't want to wait 5 minutes in order for the group to
> rebalance if I am intentionally removing a node from the cluster. I am not
> sure the best way to do this. One idea I had was adding the ability for a
> CLI or Admin API to force a rebalance of the group. This would allow for an
> admin to trigger the rebalance manually without waiting the entire
> registration timeout on shrink down. What do you think?
> >
> > Mike
> >
> > On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> >
> >     Btw, I updated KIP 345 based on my understanding. Feel free to take
> another round of look:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > KIP-345: Introduce static membership protocol to reduce ...<
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> >
> > cwiki.apache.org
> > For stateful applications, one of the biggest performance bottleneck is
> the state shuffling. In Kafka consumer, there is a concept called
> "rebalance" which means that for given M partitions and N consumers in one
> consumer group, Kafka will try to balance the load between consumers and
> ideally have ...
> >
> >
> >
> >
> >     KIP-345: Introduce static membership protocol to reduce ...<
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> >
> >     cwiki.apache.org
> >     For stateful applications, one of the biggest performance bottleneck
> is the state shuffling. In Kafka consumer, there is a concept called
> "rebalance" which means that for given M partitions and N consumers in one
> consumer group, Kafka will try to balance the load between consumers and
> ideally have ...
> >
> >
> >
> >
> >
> >     ________________________________
> >     From: Boyang Chen <bc...@outlook.com>
> >     Sent: Monday, October 29, 2018 12:34 PM
> >     To: dev@kafka.apache.org
> >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by specifying member id
> >
> >     Thanks everyone for the input on this thread! (Sorry it's been a
> while) I feel that we are very close to the final solution.
> >
> >
> >     Hey Jason and Mike, I have two quick questions on the new features
> here:
> >
> >       1.  so our proposal is that until we add a new static member into
> the group (scale up), we will not trigger rebalance until the "registration
> timeout"( the member has been offline for too long)? How about leader's
> rejoin request, I think we should still trigger rebalance when that
> happens, since the consumer group may have new topics to consume?
> >       2.  I'm not very clear on the scale up scenario in static
> membership here. Should we fallback to dynamic membership while
> adding/removing hosts (by setting member.name = null), or we still want
> to add instances with `member.name` so that we eventually expand/shrink
> the static membership? I personally feel the easier solution is to spin up
> new members and wait until either the same "registration timeout" or a
> "scale up timeout" before starting the rebalance. What do you think?
> >
> >     Meanwhile I will go ahead to make changes to the KIP with our newly
> discussed items and details. Really excited to see the design has become
> more solid.
> >
> >     Best,
> >     Boyang
> >
> >     ________________________________
> >     From: Jason Gustafson <ja...@confluent.io>
> >     Sent: Saturday, August 25, 2018 6:04 AM
> >     To: dev
> >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by specifying member id
> >
> >     Hey Mike,
> >
> >     Yeah, that's a good point. A long "registration timeout" may not be
> a great
> >     idea. Perhaps in practice you'd set it long enough to be able to
> detect a
> >     failure and provision a new instance. Maybe on the order of 10
> minutes is
> >     more reasonable.
> >
> >     In any case, it's probably a good idea to have an administrative way
> to
> >     force deregistration. One option is to extend the DeleteGroups API
> with a
> >     list of members names.
> >
> >     -Jason
> >
> >
> >
> >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> mfreyberger@appnexus.com>
> >     wrote:
> >
> >     > Jason,
> >     >
> >     > Regarding step 4 in your proposal which suggests beginning a long
> timer
> >     > (30 minutes) when a static member leaves the group, would there
> also be the
> >     > ability for an admin to force a static membership expiration?
> >     >
> >     > I’m thinking that during particular types of outages or upgrades
> users
> >     > would want forcefully remove a static member from the group.
> >     >
> >     > So the user would shut the consumer down normally, which wouldn’t
> trigger
> >     > a rebalance. Then the user could use an admin CLI tool to force
> remove that
> >     > consumer from the group, so the TopicPartitions that were
> previously owned
> >     > by that consumer can be released.
> >     >
> >     > At a high level, we need consumer groups to gracefully handle
> intermittent
> >     > failures and permanent failures. Currently, the consumer group
> protocol
> >     > handles permanent failures well, but does not handle intermittent
> failures
> >     > well (it creates unnecessary rebalances). I want to make sure the
> overall
> >     > solution here handles both intermittent failures and permanent
> failures,
> >     > rather than sacrificing support for permanent failures in order to
> provide
> >     > support for intermittent failures.
> >     >
> >     > Mike
> >     >
> >     > Sent from my iPhone
> >     >
> >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
> >     > >
> >     > > Hey Guozhang,
> >     > >
> >     > > Responses below:
> >     > >
> >     > > Originally I was trying to kill more birds with one stone with
> KIP-345,
> >     > >> e.g. to fix the multi-rebalance issue on starting up / shutting
> down a
> >     > >> multi-instance client (mentioned as case 1)/2) in my early
> email), and
> >     > >> hence proposing to have a pure static-membership protocol. But
> thinking
> >     > >> twice about it I now feel it may be too ambitious and worth
> fixing in
> >     > >> another KIP.
> >     > >
> >     > >
> >     > > I was considering an extension to support pre-initialization of
> the
> >     > static
> >     > > members of the group, but I agree we should probably leave this
> problem
> >     > for
> >     > > future work.
> >     > >
> >     > > 1. How this longish static member expiration timeout defined? Is
> it via a
> >     > >> broker, hence global config, or via a client config which can be
> >     > >> communicated to broker via JoinGroupRequest?
> >     > >
> >     > >
> >     > > I am not too sure. I tend to lean toward server-side configs
> because they
> >     > > are easier to evolve. If we have to add something to the
> protocol, then
> >     > > we'll be stuck with it forever.
> >     > >
> >     > > 2. Assuming that for static members, LEAVE_GROUP request will not
> >     > trigger a
> >     > >> rebalance immediately either, similar to session timeout, but
> only the
> >     > >> longer member expiration timeout, can we remove the internal "
> >     > >> internal.leave.group.on.close" config, which is a quick
> walk-around
> >     > then?
> >     > >
> >     > >
> >     > > Yeah, I hope we can ultimately get rid of it, but we may need it
> for
> >     > > compatibility with older brokers. A related question is what
> should be
> >     > the
> >     > > behavior of the consumer if `member.name` is provided but the
> broker
> >     > does
> >     > > not support it? We could either fail or silently downgrade to
> dynamic
> >     > > membership.
> >     > >
> >     > > -Jason
> >     > >
> >     > >
> >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> wangguoz@gmail.com>
> >     > wrote:
> >     > >>
> >     > >> Hey Jason,
> >     > >>
> >     > >> I like your idea to simplify the upgrade protocol to allow
> co-exist of
> >     > >> static and dynamic members. Admittedly it may make the
> coordinator-side
> >     > >> logic a bit more complex, but I think it worth doing it.
> >     > >>
> >     > >> Originally I was trying to kill more birds with one stone with
> KIP-345,
> >     > >> e.g. to fix the multi-rebalance issue on starting up / shutting
> down a
> >     > >> multi-instance client (mentioned as case 1)/2) in my early
> email), and
> >     > >> hence proposing to have a pure static-membership protocol. But
> thinking
> >     > >> twice about it I now feel it may be too ambitious and worth
> fixing in
> >     > >> another KIP. With that, I think what you've proposed here is a
> good way
> >     > to
> >     > >> go for KIP-345 itself.
> >     > >>
> >     > >> Note there are a few details in your proposal we'd still need
> to figure
> >     > >> out:
> >     > >>
> >     > >> 1. How this longish static member expiration timeout defined?
> Is it via
> >     > a
> >     > >> broker, hence global config, or via a client config which can be
> >     > >> communicated to broker via JoinGroupRequest?
> >     > >>
> >     > >> 2. Assuming that for static members, LEAVE_GROUP request will
> not
> >     > trigger a
> >     > >> rebalance immediately either, similar to session timeout, but
> only the
> >     > >> longer member expiration timeout, can we remove the internal "
> >     > >> internal.leave.group.on.close" config, which is a quick
> walk-around
> >     > then?
> >     > >>
> >     > >>
> >     > >>
> >     > >> Guozhang
> >     > >>
> >     > >>
> >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> jason@confluent.io>
> >     > >> wrote:
> >     > >>
> >     > >>> Hey All,
> >     > >>>
> >     > >>> Nice to see some solid progress on this. It sounds like one of
> the
> >     > >>> complications is allowing static and dynamic registration to
> coexist.
> >     > I'm
> >     > >>> wondering if we can do something like the following:
> >     > >>>
> >     > >>> 1. Statically registered members (those joining the group with
> a
> >     > >> non-null `
> >     > >>> member.name`) maintain a session with the coordinator just
> like
> >     > dynamic
> >     > >>> members.
> >     > >>> 2. If a session is active for a static member when a rebalance
> begins,
> >     > >> then
> >     > >>> basically we'll keep the current behavior. The rebalance will
> await the
> >     > >>> static member joining the group.
> >     > >>> 3. If a static member does not have an active session, then the
> >     > >> coordinator
> >     > >>> will not wait for it to join, but will still include it in the
> >     > rebalance.
> >     > >>> The coordinator will forward the cached subscription
> information to the
> >     > >>> leader and will cache the assignment after the rebalance
> completes.
> >     > (Note
> >     > >>> that we still have the generationId to fence offset commits
> from a
> >     > static
> >     > >>> zombie if the assignment changes.)
> >     > >>> 4. When a static member leaves the group or has its session
> expire, no
> >     > >>> rebalance is triggered. Instead, we can begin a timer to
> expire the
> >     > >> static
> >     > >>> registration. This would be a longish timeout (like 30 minutes
> say).
> >     > >>>
> >     > >>> So basically static members participate in all rebalances
> regardless
> >     > >>> whether they have an active session. In a given rebalance,
> some of the
> >     > >>> members may be static and some dynamic. The group leader can
> >     > >> differentiate
> >     > >>> the two based on the presence of the `member.name` (we have
> to add
> >     > this
> >     > >> to
> >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> leaders
> >     > >>> preferentially from the active members that support the latest
> >     > JoinGroup
> >     > >>> protocol and are using static membership. If we have to choose
> a leader
> >     > >>> with an old version, however, it would see all members in the
> group
> >     > >> (static
> >     > >>> or dynamic) as dynamic members and perform the assignment as
> usual.
> >     > >>>
> >     > >>> Would that work?
> >     > >>>
> >     > >>> -Jason
> >     > >>>
> >     > >>>
> >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> wangguoz@gmail.com>
> >     > >> wrote:
> >     > >>>
> >     > >>>> Hello Boyang,
> >     > >>>>
> >     > >>>> Thanks for the updated proposal, a few questions:
> >     > >>>>
> >     > >>>> 1. Where will "change-group-timeout" be communicated to the
> broker?
> >     > >> Will
> >     > >>>> that be a new field in the JoinGroupRequest, or are we going
> to
> >     > >>> piggy-back
> >     > >>>> on the existing session-timeout field (assuming that the
> original
> >     > value
> >     > >>>> will not be used anywhere in the static membership any more)?
> >     > >>>>
> >     > >>>> 2. "However, if the consumer takes longer than session
> timeout to
> >     > >> return,
> >     > >>>> we shall still trigger rebalance but it could still try to
> catch
> >     > >>>> `change-group-timeout`.": what does this mean? I thought your
> proposal
> >     > >> is
> >     > >>>> that for static memberships, the broker will NOT trigger
> rebalance
> >     > even
> >     > >>>> after session-timeout has been detected, but only that after
> >     > >>>> change-group-timeout
> >     > >>>> which is supposed to be longer than session-timeout to be
> defined?
> >     > >>>>
> >     > >>>> 3. "A join group request with member.name set will be
> treated as
> >     > >>>> `static-membership` strategy", in this case, how would the
> switch from
> >     > >>>> dynamic to static happen, since whoever changed the
> member.name to
> >     > >>>> not-null
> >     > >>>> will be rejected, right?
> >     > >>>>
> >     > >>>> 4. "just erase the cached mapping, and wait for session
> timeout to
> >     > >>> trigger
> >     > >>>> rebalance should be sufficient." this is also a bit unclear
> to me: who
> >     > >>> will
> >     > >>>> erase the cached mapping? Since it is on the broker-side I
> assume that
> >     > >>>> broker has to do it. Are you suggesting to use a new request
> for it?
> >     > >>>>
> >     > >>>> 5. "Halfway switch": following 3) above, if your proposal is
> basically
> >     > >> to
> >     > >>>> let "first join-request wins", and the strategy will stay as
> is until
> >     > >> all
> >     > >>>> members are gone, then this will also not happen since
> whoever used
> >     > >>>> different strategy as the first guy who sends join-group
> request will
> >     > >> be
> >     > >>>> rejected right?
> >     > >>>>
> >     > >>>>
> >     > >>>> Guozhang
> >     > >>>>
> >     > >>>>
> >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> john@confluent.io>
> >     > >> wrote:
> >     > >>>>
> >     > >>>>> This sounds good to me!
> >     > >>>>>
> >     > >>>>> Thanks for the time you've spent on it,
> >     > >>>>> -John
> >     > >>>>>
> >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> bchen11@outlook.com>
> >     > >>>> wrote:
> >     > >>>>>
> >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
> >     > >> haven't
> >     > >>>> got
> >     > >>>>>> time to update this thread. To summarize what we come up so
> far,
> >     > >> here
> >     > >>>> is
> >     > >>>>> a
> >     > >>>>>> draft updated plan:
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Introduce a new config called `member.name` which is
> supposed to
> >     > >> be
> >     > >>>>>> provided uniquely by the consumer client. The broker will
> maintain
> >     > >> a
> >     > >>>>> cache
> >     > >>>>>> with [key:member.name, value:member.id]. A join group
> request with
> >     > >>>>>> member.name set will be treated as `static-membership`
> strategy,
> >     > >> and
> >     > >>>>> will
> >     > >>>>>> reject any join group request without member.name. So this
> >     > >>>> coordination
> >     > >>>>>> change will be differentiated from the `dynamic-membership`
> >     > >> protocol
> >     > >>> we
> >     > >>>>>> currently have.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When handling static join group request:
> >     > >>>>>>
> >     > >>>>>>  1.   The broker will check the membership to see whether
> this is
> >     > >> a
> >     > >>>> new
> >     > >>>>>> member. If new, broker allocate a unique member id, cache
> the
> >     > >> mapping
> >     > >>>> and
> >     > >>>>>> move to rebalance stage.
> >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> will not
> >     > >>>> change
> >     > >>>>>> group state, and return its cached member.id and current
> >     > >> assignment.
> >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with
> pair
> >     > >>> member
> >     > >>>>>> name and id, I think for join group request it is ok to
> leave
> >     > >> member
> >     > >>> id
> >     > >>>>>> blank as member name is the unique identifier. In commit
> offset
> >     > >>> request
> >     > >>>>> we
> >     > >>>>>> *must* have both.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When handling commit offset request, if enabled with static
> >     > >>> membership,
> >     > >>>>>> each time the commit request must have both member.name and
> >     > >>> member.id
> >     > >>>> to
> >     > >>>>>> be identified as a `certificated member`. If not, this
> means there
> >     > >>> are
> >     > >>>>>> duplicate consumer members with same member name and the
> request
> >     > >> will
> >     > >>>> be
> >     > >>>>>> rejected to guarantee consumption uniqueness.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When rolling restart/shutting down gracefully, the client
> will
> >     > >> send a
> >     > >>>>>> leave group request (static membership mode). In static
> membership,
> >     > >>> we
> >     > >>>>> will
> >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> provided by
> >     > >>>>> leader.
> >     > >>>>>> So we will wait for all the members to rejoin the group and
> do
> >     > >>> exactly
> >     > >>>>> one
> >     > >>>>>> rebalance since all members are expected to rejoin within
> timeout.
> >     > >> If
> >     > >>>>>> consumer crashes, the join group request from the restarted
> >     > >> consumer
> >     > >>>> will
> >     > >>>>>> be recognized as an existing member and be handled as above
> >     > >> condition
> >     > >>>> 1;
> >     > >>>>>> However, if the consumer takes longer than session timeout
> to
> >     > >> return,
> >     > >>>> we
> >     > >>>>>> shall still trigger rebalance but it could still try to
> catch
> >     > >>>>>> `change-group-timeout`. If it failed to catch second
> timeout, its
> >     > >>>> cached
> >     > >>>>>> state on broker will be garbage collected and trigger a new
> >     > >> rebalance
> >     > >>>>> when
> >     > >>>>>> it finally joins.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> And consider the switch between dynamic to static
> membership.
> >     > >>>>>>
> >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
> >     > >> membership
> >     > >>>> to
> >     > >>>>>> static and wait for all the current members to restart,
> since their
> >     > >>>>>> membership is still dynamic. Here our assumption is that the
> >     > >> restart
> >     > >>>>>> process shouldn't take a long time, as long restart is
> breaking the
> >     > >>>>>> `rebalance timeout` in whatever membership protocol we are
> using.
> >     > >>>> Before
> >     > >>>>>> restart, all dynamic member join requests will be rejected.
> >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade which
> should
> >     > >>> be
> >     > >>>>>> smooth: just erase the cached mapping, and wait for session
> timeout
> >     > >>> to
> >     > >>>>>> trigger rebalance should be sufficient. (Fallback to current
> >     > >>> behavior)
> >     > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
> >     > >> dynamic
> >     > >>>>>> membership while some keep static membership. This will
> cause the
> >     > >>> group
> >     > >>>>>> rebalance forever without progress because dynamic/static
> states
> >     > >> are
> >     > >>>>>> bouncing each other. This could guarantee that we will not
> make the
> >     > >>>>>> consumer group work in a wrong state by having half static
> and half
> >     > >>>>> dynamic.
> >     > >>>>>>
> >     > >>>>>> To guarantee correctness, we will also push the member
> name/id pair
> >     > >>> to
> >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> upgrade the
> >     > >> API
> >     > >>>>>> version, these details will be further discussed back in
> the KIP.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Are there any concern for this high level proposal? Just
> want to
> >     > >>>>> reiterate
> >     > >>>>>> on the core idea of the KIP: "If the broker recognize this
> consumer
> >     > >>> as
> >     > >>>> an
> >     > >>>>>> existing member, it shouldn't trigger rebalance".
> >     > >>>>>>
> >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is
> much
> >     > >> more
> >     > >>>>>> robust than previous one!
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Best,
> >     > >>>>>>
> >     > >>>>>> Boyang
> >     > >>>>>>
> >     > >>>>>> ________________________________
> >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> >     > >>>>>> To: dev@kafka.apache.org
> >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> >     > >>> by
> >     > >>>>>> specifying member id
> >     > >>>>>>
> >     > >>>>>> Hi,
> >     > >>>>>>
> >     > >>>>>> thanks for the detailed discussion. I learned a lot about
> internals
> >     > >>>> again
> >     > >>>>>> :)
> >     > >>>>>>
> >     > >>>>>> I like the idea or a user config `member.name` and to keep
> `
> >     > >>> member.id`
> >     > >>>>>> internal. Also agree with Guozhang, that reusing `client.id`
> might
> >     > >>> not
> >     > >>>>>> be a good idea.
> >     > >>>>>>
> >     > >>>>>> To clarify the algorithm, each time we generate a new `
> member.id`,
> >     > >>> we
> >     > >>>>>> also need to update the "group membership" information (ie,
> mapping
> >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id`
> replaces
> >     > >>> the
> >     > >>>>>> old entry in the cache.
> >     > >>>>>>
> >     > >>>>>> I also think, we need to preserve the `member.name ->
> member.id`
> >     > >>>> mapping
> >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> this IMHO.
> >     > >>>>>>
> >     > >>>>>> For changing the default value of config
> `leave.group.on.close`. I
> >     > >>>> agree
> >     > >>>>>> with John, that we should not change the default config,
> because it
> >     > >>>>>> would impact all consumer groups with dynamic assignment.
> However,
> >     > >> I
> >     > >>>>>> think we can document, that if static assignment is used
> (ie,
> >     > >>>>>> `member.name` is configured) we never send a
> LeaveGroupRequest
> >     > >>>>>> regardless of the config. Note, that the config is
> internal, so not
> >     > >>>> sure
> >     > >>>>>> how to document this in detail. We should not expose the
> internal
> >     > >>>> config
> >     > >>>>>> in the docs.
> >     > >>>>>>
> >     > >>>>>> About upgrading: why do we need have two rolling bounces
> and encode
> >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> >     > >>>>>>
> >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> static, I
> >     > >>>> don't
> >     > >>>>>> see any reason why both should not work together and single
> rolling
> >     > >>>>>> bounce would not be sufficient? If we bounce the first
> consumer and
> >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> and the
> >     > >>> broker
> >     > >>>>>> registers the [member.name, member.id] in the cache. Why
> would
> >     > >> this
> >     > >>>>>> interfere with all other consumer that use dynamic
> assignment?
> >     > >>>>>>
> >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> need to
> >     > >> check
> >     > >>>> if
> >     > >>>>>> the mapping [member.name, member.id] contains the send `
> member.id`
> >     > >>> --
> >     > >>>> I
> >     > >>>>>> don't think this is necessary -- it seems to be sufficient
> to check
> >     > >>> the
> >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as be
> do
> >     > >> today
> >     > >>> --
> >     > >>>>>> thus, checking `member.id` does not require any change
> IMHO.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> -Matthias
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> >     > >>>>>>> @James
> >     > >>>>>>>
> >     > >>>>>>> What you described is true: the transition from dynamic to
> static
> >     > >>>>>>> memberships are not thought through yet. But I do not
> think it is
> >     > >>> an
> >     > >>>>>>> impossible problem: note that we indeed moved the offset
> commit
> >     > >>> from
> >     > >>>> ZK
> >     > >>>>>> to
> >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> first to
> >     > >>>>>>> double-commits on both zk and coordinator, and then do a
> second
> >     > >>> round
> >     > >>>>> to
> >     > >>>>>>> turn the zk off.
> >     > >>>>>>>
> >     > >>>>>>> So just to throw a wild idea here: also following a
> >     > >>>> two-rolling-bounce
> >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> "static"
> >     > >>> while
> >     > >>>>>> keep
> >     > >>>>>>> the registry-id field empty still, in this case, the
> coordinator
> >     > >>>> still
> >     > >>>>>>> follows the logic of "dynamic", accepting the request while
> >     > >>> allowing
> >     > >>>>> the
> >     > >>>>>>> protocol to be set to "static"; after the first rolling
> bounce,
> >     > >> the
> >     > >>>>> group
> >     > >>>>>>> protocol is already "static", then a second rolling bounce
> is
> >     > >>>> triggered
> >     > >>>>>> and
> >     > >>>>>>> this time we set the registry-id.
> >     > >>>>>>>
> >     > >>>>>>>
> >     > >>>>>>> Guozhang
> >     > >>>>>>>
> >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> >     > >> wushujames@gmail.com>
> >     > >>>>>> wrote:
> >     > >>>>>>>
> >     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
> >     > >>>>>>>>
> >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> wangguoz@gmail.com
> >     > >>>
> >     > >>>>> wrote:
> >     > >>>>>>>>>
> >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> fields:
> >     > >>>>>>>>>
> >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
> >     > >>> protocols.
> >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> pre-defined
> >     > >>> member
> >     > >>>>> id.
> >     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
> >     > >>>>>>>>> "group-change-timeout" value.
> >     > >>>>>>>>>
> >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> protocols
> >     > >>> for
> >     > >>>>> all
> >     > >>>>>>>>> group members: we accept the protocol on the first joined
> >     > >> member
> >     > >>> of
> >     > >>>>> the
> >     > >>>>>>>>> group, and if later joining members indicate a different
> >     > >>> membership
> >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> value was
> >     > >>>>> different
> >     > >>>>>>>> to
> >     > >>>>>>>>> the first joined member, we reject it as well.
> >     > >>>>>>>>
> >     > >>>>>>>>
> >     > >>>>>>>> What will happen if we have an already-deployed
> application that
> >     > >>>> wants
> >     > >>>>>> to
> >     > >>>>>>>> switch to using static membership? Let’s say there are 10
> >     > >>> instances
> >     > >>>> of
> >     > >>>>>> it.
> >     > >>>>>>>> As the instances go through a rolling restart, they will
> switch
> >     > >>> from
> >     > >>>>>>>> dynamic membership (the default?) to static membership.
> As each
> >     > >>> one
> >     > >>>>>> leaves
> >     > >>>>>>>> the group and restarts, they will be rejected from the
> group
> >     > >>>> (because
> >     > >>>>>> the
> >     > >>>>>>>> group is currently using dynamic membership). The group
> will
> >     > >>> shrink
> >     > >>>>> down
> >     > >>>>>>>> until there is 1 node handling all the traffic. After
> that one
> >     > >>>>> restarts,
> >     > >>>>>>>> the group will switch over to static membership.
> >     > >>>>>>>>
> >     > >>>>>>>> Is that right? That means that the transition plan from
> dynamic
> >     > >> to
> >     > >>>>>> static
> >     > >>>>>>>> membership isn’t very smooth.
> >     > >>>>>>>>
> >     > >>>>>>>> I’m not really sure what can be done in this case. This
> reminds
> >     > >> me
> >     > >>>> of
> >     > >>>>>> the
> >     > >>>>>>>> transition plans that were discussed for moving from
> >     > >>> zookeeper-based
> >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was
> also
> >     > >>> hard,
> >     > >>>>> and
> >     > >>>>>>>> ultimately we decided not to build that.
> >     > >>>>>>>>
> >     > >>>>>>>> -James
> >     > >>>>>>>>
> >     > >>>>>>>>
> >     > >>>>>>>
> >     > >>>>>>>
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>
> >     > >>>>
> >     > >>>>
> >     > >>>>
> >     > >>>> --
> >     > >>>> -- Guozhang
> >     > >>>>
> >     > >>>
> >     > >>
> >     > >>
> >     > >>
> >     > >> --
> >     > >> -- Guozhang
> >     > >>
> >     >
> >
> >
>
>

--
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Mayuresh Gharat <gh...@gmail.com>.
Hi Boyang,

Thanks for the KIP and sorry for being late to the party. This KIP is
really useful for us at Linkedin.

I had a few questions :

The idea of having static member name seems nice, but instead of a config,
would it be possible for it to be passed in to the consumer at runtime?
This is because an app might want to decide the config value at runtime
using its host information for example, to generate the unique member name.

Also the KIP talks about using the "REGISTRATION_TIMEOUT_MS". I was
wondering if we can reuse the session timeout here. This might help us to
have one less config on the consumer.

The KIP also talks about adding another internal topic "static_member_map".
Would the semantics (GroupCoordinator broker, topic configs) be the same as
__consumer_offsets topic?

Thanks,

Mayuresh


On Wed, Nov 7, 2018 at 12:17 AM Boyang Chen <bc...@outlook.com> wrote:

> I took a quick pass of the proposal. First I would say it's a very
> brilliant initiative from Konstantine and Confluent folks. To draft up a
> proposal like this needs deep understanding of the rebalance protocol! I
> summarized some thoughts here.
>
>
> Overall the motivations of the two proposals align on that:
>
>   1.  Both believe the invariant resource (belonging to the same process)
> should be preserved across rebalance.
>   2.  Transit failures (K8 thread death) shouldn't trigger resource
> redistribution. I don't use rebalance here since part one of the
> cooperative proposal could potentially introduce more rebalances but only
> on must-move resources.
>   3.  Scale up/down and rolling bounce are causing unnecessary resource
> shuffling that need to be mitigated.
>
>
> On motivation level, I think both approach could solve/mitigate the above
> issues. They are just different in design philosophy, or I would say the
> perspective difference between framework user and algorithm designer.
>
>
> Two proposals have different focuses. KIP-345 is trying to place more
> fine-grained control on the broker side to reduce the unnecessary
> rebalances, while keeping the client logic intact. This is pretty intuitive
> cause-effect for normal developers who are not very familiar with rebalance
> protocol. As a developer working with Kafka Streams daily, I'd be happy to
> see a simplified rebalance protocol and just focus on maintaining the
> stream/consumer jobs. Too many rebalances raised my concern on the job
> health. To be concise, static membership has the advantage of reducing
> mental burden.
>
>
> Cooperative proposal takes thoughtful approach on client side. We want to
> have fine-grained control on the join/exit group behaviors and make the
> current dynamic membership better to address above issues. I do feel our
> idea crossed on the delayed rebalance when we scale up/down, which could
> potentially reduce the state shuffling and decouple the behavior from
> session timeout which is already overloaded.  In this sense, I believe both
> approaches would serve well in making "reasonable rebalance" happen at the
> "right timing".
>
>
> However, based on my understanding, either 345 or cooperative rebalancing
> is not solving the problem Mike has proposed: could we do a better job at
> scaling up/down in ideal timing? My initial response was to introduce an
> admin API which now I feel is sub-optimal, in that the goal of smooth
> transition is to make sure the newly up hosts are actually "ready". For
> example:
>
>
> We have 4 instance reading from 8 topic partitions (= 8 tasks). At some
> time we would like to scale up to 8 hosts, with the current improvements we
> could reduce 4 potential rebalances to a single one. But the new hosts are
> yet unknown to be "ready" if they need to reconstruct the local state. To
> be actually ready, we need 4 standby tasks running on those empty hosts and
> leader needs to wait for the signal of "replay/reconstruct complete" to
> actually involve them into the main consumer group. Otherwise, rebalance
> just kills our performance since we need to wait indefinite long for task
> migration.
>
>
> The scale down is also tricky such that we are not able to define a "true"
> leave of a member. Rebalance immediately after "true" leaves are most
> optimal comparing with human intervention. Does this make sense?
>
>
> My intuition is that cooperative approach which was implemented on the
> client side could better handle scaling cases than KIP 345, since it
> involves a lot of algorithmic changes to define "replaying" stage, which I
> feel would over-complicate broker logic if implemented on coordinator. If
> we let 345 focus on reducing unnecessary rebalance, and let cooperative
> approach focus on judging best timing of scale up/down, the two efforts
> could be aligned. In long term, I feel the more complex improvement of
> consumer protocol should happen on client side instead of server side which
> is easier to test and has less global impact for the entire Kafka
> production cluster.
>
>
> Thanks again to Konstantine, Matthias and other folks in coming up with
> this great client proposal. This is great complementation to KIP 345. In a
> high level, we are not having any collision on the path and both proposals
> are making sense here. Just need better sync to avoid duplicate effort :)
>
>
> Best,
>
> Boyang
>
>
> ________________________________
> From: Boyang Chen <bc...@outlook.com>
> Sent: Wednesday, November 7, 2018 1:57 PM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Thanks Matthias for bringing this awesome proposal up! I shall take a
> deeper look and make a comparison between the two proposals.
>
>
> Meanwhile for the scale down specifically for stateful streaming, we could
> actually introduce a new status called "learner" where the newly up hosts
> could try to catch up with the assigned task progress first before
> triggering the rebalance, from which we don't see a sudden dip on the
> progress. However, it is built on top of the success of KIP-345.
>
>
> ________________________________
> From: Matthias J. Sax <ma...@confluent.io>
> Sent: Wednesday, November 7, 2018 7:02 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hey,
>
> there was quite a pause on this KIP discussion and in the mean time, a
> new design for incremental cooporative rebalance was suggested:
>
>
> https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies
>
>
> We should make sure that the proposal and this KIP align to each other.
> Thoughts?
>
>
> -Matthias
>
> On 11/5/18 7:31 PM, Boyang Chen wrote:
> > Hey Mike,
> >
> >
> > thanks for the feedback, the two question are very thoughtful!
> >
> >
> >> 1) I am a little confused about the distinction for the leader. If the
> consumer node that was assigned leader does a bounce (goes down and quickly
> comes up) to update application code, will a rebalance be triggered? I > do
> not think a bounce of the leader should trigger a rebalance.
> >
> > For Q1 my intention was to minimize the change within one KIP, since the
> leader rejoining case could be addressed separately.
> >
> >
> >> 2) The timeout for shrink up makes a lot of sense and allows to
> gracefully increase the number of nodes in the cluster. I think we need to
> support graceful shrink down as well. If I set the registration timeout to
> 5 minutes > to handle rolling restarts or intermittent failures without
> shuffling state, I don't want to wait 5 minutes in order for the group to
> rebalance if I am intentionally removing a node from the cluster. I am not
> sure the best way to > do this. One idea I had was adding the ability for a
> CLI or Admin API to force a rebalance of the group. This would allow for an
> admin to trigger the rebalance manually without waiting the entire
> registration timeout on > shrink down. What do you think?
> >
> > For 2) my understanding is that for scaling down case it is better to be
> addressed by CLI tool than code logic, since only by human evaluation we
> could decide whether it is a "right timing" -- the time when all the
> scaling down consumers are offline -- to kick in rebalance. Unless we
> introduce another term on coordinator which indicates the target consumer
> group size, broker will find it hard to decide when to start rebalance. So
> far I prefer to hold the implementation for that, but agree we could
> discuss whether we want to introduce admin API in this KIP or a separate
> one.
> >
> >
> > Thanks again for the proposed ideas!
> >
> >
> > Boyang
> >
> > ________________________________
> > From: Mike Freyberger <mi...@xandr.com>
> > Sent: Monday, November 5, 2018 6:13 AM
> > To: dev@kafka.apache.org
> > Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
> >
> > Boyang,
> >
> > Thanks for updating the KIP. It's shaping up well. Two things:
> >
> > 1) I am a little confused about the distinction for the leader. If the
> consumer node that was assigned leader does a bounce (goes down and quickly
> comes up) to update application code, will a rebalance be triggered? I do
> not think a bounce of the leader should trigger a rebalance.
> >
> > 2) The timeout for shrink up makes a lot of sense and allows to
> gracefully increase the number of nodes in the cluster. I think we need to
> support graceful shrink down as well. If I set the registration timeout to
> 5 minutes to handle rolling restarts or intermittent failures without
> shuffling state, I don't want to wait 5 minutes in order for the group to
> rebalance if I am intentionally removing a node from the cluster. I am not
> sure the best way to do this. One idea I had was adding the ability for a
> CLI or Admin API to force a rebalance of the group. This would allow for an
> admin to trigger the rebalance manually without waiting the entire
> registration timeout on shrink down. What do you think?
> >
> > Mike
> >
> > On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> >
> >     Btw, I updated KIP 345 based on my understanding. Feel free to take
> another round of look:
> >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> > KIP-345: Introduce static membership protocol to reduce ...<
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> >
> > cwiki.apache.org
> > For stateful applications, one of the biggest performance bottleneck is
> the state shuffling. In Kafka consumer, there is a concept called
> "rebalance" which means that for given M partitions and N consumers in one
> consumer group, Kafka will try to balance the load between consumers and
> ideally have ...
> >
> >
> >
> >
> >     KIP-345: Introduce static membership protocol to reduce ...<
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> >
> >     cwiki.apache.org
> >     For stateful applications, one of the biggest performance bottleneck
> is the state shuffling. In Kafka consumer, there is a concept called
> "rebalance" which means that for given M partitions and N consumers in one
> consumer group, Kafka will try to balance the load between consumers and
> ideally have ...
> >
> >
> >
> >
> >
> >     ________________________________
> >     From: Boyang Chen <bc...@outlook.com>
> >     Sent: Monday, October 29, 2018 12:34 PM
> >     To: dev@kafka.apache.org
> >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by specifying member id
> >
> >     Thanks everyone for the input on this thread! (Sorry it's been a
> while) I feel that we are very close to the final solution.
> >
> >
> >     Hey Jason and Mike, I have two quick questions on the new features
> here:
> >
> >       1.  so our proposal is that until we add a new static member into
> the group (scale up), we will not trigger rebalance until the "registration
> timeout"( the member has been offline for too long)? How about leader's
> rejoin request, I think we should still trigger rebalance when that
> happens, since the consumer group may have new topics to consume?
> >       2.  I'm not very clear on the scale up scenario in static
> membership here. Should we fallback to dynamic membership while
> adding/removing hosts (by setting member.name = null), or we still want
> to add instances with `member.name` so that we eventually expand/shrink
> the static membership? I personally feel the easier solution is to spin up
> new members and wait until either the same "registration timeout" or a
> "scale up timeout" before starting the rebalance. What do you think?
> >
> >     Meanwhile I will go ahead to make changes to the KIP with our newly
> discussed items and details. Really excited to see the design has become
> more solid.
> >
> >     Best,
> >     Boyang
> >
> >     ________________________________
> >     From: Jason Gustafson <ja...@confluent.io>
> >     Sent: Saturday, August 25, 2018 6:04 AM
> >     To: dev
> >     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
> by specifying member id
> >
> >     Hey Mike,
> >
> >     Yeah, that's a good point. A long "registration timeout" may not be
> a great
> >     idea. Perhaps in practice you'd set it long enough to be able to
> detect a
> >     failure and provision a new instance. Maybe on the order of 10
> minutes is
> >     more reasonable.
> >
> >     In any case, it's probably a good idea to have an administrative way
> to
> >     force deregistration. One option is to extend the DeleteGroups API
> with a
> >     list of members names.
> >
> >     -Jason
> >
> >
> >
> >     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <
> mfreyberger@appnexus.com>
> >     wrote:
> >
> >     > Jason,
> >     >
> >     > Regarding step 4 in your proposal which suggests beginning a long
> timer
> >     > (30 minutes) when a static member leaves the group, would there
> also be the
> >     > ability for an admin to force a static membership expiration?
> >     >
> >     > I’m thinking that during particular types of outages or upgrades
> users
> >     > would want forcefully remove a static member from the group.
> >     >
> >     > So the user would shut the consumer down normally, which wouldn’t
> trigger
> >     > a rebalance. Then the user could use an admin CLI tool to force
> remove that
> >     > consumer from the group, so the TopicPartitions that were
> previously owned
> >     > by that consumer can be released.
> >     >
> >     > At a high level, we need consumer groups to gracefully handle
> intermittent
> >     > failures and permanent failures. Currently, the consumer group
> protocol
> >     > handles permanent failures well, but does not handle intermittent
> failures
> >     > well (it creates unnecessary rebalances). I want to make sure the
> overall
> >     > solution here handles both intermittent failures and permanent
> failures,
> >     > rather than sacrificing support for permanent failures in order to
> provide
> >     > support for intermittent failures.
> >     >
> >     > Mike
> >     >
> >     > Sent from my iPhone
> >     >
> >     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io>
> wrote:
> >     > >
> >     > > Hey Guozhang,
> >     > >
> >     > > Responses below:
> >     > >
> >     > > Originally I was trying to kill more birds with one stone with
> KIP-345,
> >     > >> e.g. to fix the multi-rebalance issue on starting up / shutting
> down a
> >     > >> multi-instance client (mentioned as case 1)/2) in my early
> email), and
> >     > >> hence proposing to have a pure static-membership protocol. But
> thinking
> >     > >> twice about it I now feel it may be too ambitious and worth
> fixing in
> >     > >> another KIP.
> >     > >
> >     > >
> >     > > I was considering an extension to support pre-initialization of
> the
> >     > static
> >     > > members of the group, but I agree we should probably leave this
> problem
> >     > for
> >     > > future work.
> >     > >
> >     > > 1. How this longish static member expiration timeout defined? Is
> it via a
> >     > >> broker, hence global config, or via a client config which can be
> >     > >> communicated to broker via JoinGroupRequest?
> >     > >
> >     > >
> >     > > I am not too sure. I tend to lean toward server-side configs
> because they
> >     > > are easier to evolve. If we have to add something to the
> protocol, then
> >     > > we'll be stuck with it forever.
> >     > >
> >     > > 2. Assuming that for static members, LEAVE_GROUP request will not
> >     > trigger a
> >     > >> rebalance immediately either, similar to session timeout, but
> only the
> >     > >> longer member expiration timeout, can we remove the internal "
> >     > >> internal.leave.group.on.close" config, which is a quick
> walk-around
> >     > then?
> >     > >
> >     > >
> >     > > Yeah, I hope we can ultimately get rid of it, but we may need it
> for
> >     > > compatibility with older brokers. A related question is what
> should be
> >     > the
> >     > > behavior of the consumer if `member.name` is provided but the
> broker
> >     > does
> >     > > not support it? We could either fail or silently downgrade to
> dynamic
> >     > > membership.
> >     > >
> >     > > -Jason
> >     > >
> >     > >
> >     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <
> wangguoz@gmail.com>
> >     > wrote:
> >     > >>
> >     > >> Hey Jason,
> >     > >>
> >     > >> I like your idea to simplify the upgrade protocol to allow
> co-exist of
> >     > >> static and dynamic members. Admittedly it may make the
> coordinator-side
> >     > >> logic a bit more complex, but I think it worth doing it.
> >     > >>
> >     > >> Originally I was trying to kill more birds with one stone with
> KIP-345,
> >     > >> e.g. to fix the multi-rebalance issue on starting up / shutting
> down a
> >     > >> multi-instance client (mentioned as case 1)/2) in my early
> email), and
> >     > >> hence proposing to have a pure static-membership protocol. But
> thinking
> >     > >> twice about it I now feel it may be too ambitious and worth
> fixing in
> >     > >> another KIP. With that, I think what you've proposed here is a
> good way
> >     > to
> >     > >> go for KIP-345 itself.
> >     > >>
> >     > >> Note there are a few details in your proposal we'd still need
> to figure
> >     > >> out:
> >     > >>
> >     > >> 1. How this longish static member expiration timeout defined?
> Is it via
> >     > a
> >     > >> broker, hence global config, or via a client config which can be
> >     > >> communicated to broker via JoinGroupRequest?
> >     > >>
> >     > >> 2. Assuming that for static members, LEAVE_GROUP request will
> not
> >     > trigger a
> >     > >> rebalance immediately either, similar to session timeout, but
> only the
> >     > >> longer member expiration timeout, can we remove the internal "
> >     > >> internal.leave.group.on.close" config, which is a quick
> walk-around
> >     > then?
> >     > >>
> >     > >>
> >     > >>
> >     > >> Guozhang
> >     > >>
> >     > >>
> >     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <
> jason@confluent.io>
> >     > >> wrote:
> >     > >>
> >     > >>> Hey All,
> >     > >>>
> >     > >>> Nice to see some solid progress on this. It sounds like one of
> the
> >     > >>> complications is allowing static and dynamic registration to
> coexist.
> >     > I'm
> >     > >>> wondering if we can do something like the following:
> >     > >>>
> >     > >>> 1. Statically registered members (those joining the group with
> a
> >     > >> non-null `
> >     > >>> member.name`) maintain a session with the coordinator just
> like
> >     > dynamic
> >     > >>> members.
> >     > >>> 2. If a session is active for a static member when a rebalance
> begins,
> >     > >> then
> >     > >>> basically we'll keep the current behavior. The rebalance will
> await the
> >     > >>> static member joining the group.
> >     > >>> 3. If a static member does not have an active session, then the
> >     > >> coordinator
> >     > >>> will not wait for it to join, but will still include it in the
> >     > rebalance.
> >     > >>> The coordinator will forward the cached subscription
> information to the
> >     > >>> leader and will cache the assignment after the rebalance
> completes.
> >     > (Note
> >     > >>> that we still have the generationId to fence offset commits
> from a
> >     > static
> >     > >>> zombie if the assignment changes.)
> >     > >>> 4. When a static member leaves the group or has its session
> expire, no
> >     > >>> rebalance is triggered. Instead, we can begin a timer to
> expire the
> >     > >> static
> >     > >>> registration. This would be a longish timeout (like 30 minutes
> say).
> >     > >>>
> >     > >>> So basically static members participate in all rebalances
> regardless
> >     > >>> whether they have an active session. In a given rebalance,
> some of the
> >     > >>> members may be static and some dynamic. The group leader can
> >     > >> differentiate
> >     > >>> the two based on the presence of the `member.name` (we have
> to add
> >     > this
> >     > >> to
> >     > >>> the JoinGroupResponse). Generally speaking, we would choose
> leaders
> >     > >>> preferentially from the active members that support the latest
> >     > JoinGroup
> >     > >>> protocol and are using static membership. If we have to choose
> a leader
> >     > >>> with an old version, however, it would see all members in the
> group
> >     > >> (static
> >     > >>> or dynamic) as dynamic members and perform the assignment as
> usual.
> >     > >>>
> >     > >>> Would that work?
> >     > >>>
> >     > >>> -Jason
> >     > >>>
> >     > >>>
> >     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <
> wangguoz@gmail.com>
> >     > >> wrote:
> >     > >>>
> >     > >>>> Hello Boyang,
> >     > >>>>
> >     > >>>> Thanks for the updated proposal, a few questions:
> >     > >>>>
> >     > >>>> 1. Where will "change-group-timeout" be communicated to the
> broker?
> >     > >> Will
> >     > >>>> that be a new field in the JoinGroupRequest, or are we going
> to
> >     > >>> piggy-back
> >     > >>>> on the existing session-timeout field (assuming that the
> original
> >     > value
> >     > >>>> will not be used anywhere in the static membership any more)?
> >     > >>>>
> >     > >>>> 2. "However, if the consumer takes longer than session
> timeout to
> >     > >> return,
> >     > >>>> we shall still trigger rebalance but it could still try to
> catch
> >     > >>>> `change-group-timeout`.": what does this mean? I thought your
> proposal
> >     > >> is
> >     > >>>> that for static memberships, the broker will NOT trigger
> rebalance
> >     > even
> >     > >>>> after session-timeout has been detected, but only that after
> >     > >>>> change-group-timeout
> >     > >>>> which is supposed to be longer than session-timeout to be
> defined?
> >     > >>>>
> >     > >>>> 3. "A join group request with member.name set will be
> treated as
> >     > >>>> `static-membership` strategy", in this case, how would the
> switch from
> >     > >>>> dynamic to static happen, since whoever changed the
> member.name to
> >     > >>>> not-null
> >     > >>>> will be rejected, right?
> >     > >>>>
> >     > >>>> 4. "just erase the cached mapping, and wait for session
> timeout to
> >     > >>> trigger
> >     > >>>> rebalance should be sufficient." this is also a bit unclear
> to me: who
> >     > >>> will
> >     > >>>> erase the cached mapping? Since it is on the broker-side I
> assume that
> >     > >>>> broker has to do it. Are you suggesting to use a new request
> for it?
> >     > >>>>
> >     > >>>> 5. "Halfway switch": following 3) above, if your proposal is
> basically
> >     > >> to
> >     > >>>> let "first join-request wins", and the strategy will stay as
> is until
> >     > >> all
> >     > >>>> members are gone, then this will also not happen since
> whoever used
> >     > >>>> different strategy as the first guy who sends join-group
> request will
> >     > >> be
> >     > >>>> rejected right?
> >     > >>>>
> >     > >>>>
> >     > >>>> Guozhang
> >     > >>>>
> >     > >>>>
> >     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <
> john@confluent.io>
> >     > >> wrote:
> >     > >>>>
> >     > >>>>> This sounds good to me!
> >     > >>>>>
> >     > >>>>> Thanks for the time you've spent on it,
> >     > >>>>> -John
> >     > >>>>>
> >     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <
> bchen11@outlook.com>
> >     > >>>> wrote:
> >     > >>>>>
> >     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
> >     > >> haven't
> >     > >>>> got
> >     > >>>>>> time to update this thread. To summarize what we come up so
> far,
> >     > >> here
> >     > >>>> is
> >     > >>>>> a
> >     > >>>>>> draft updated plan:
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Introduce a new config called `member.name` which is
> supposed to
> >     > >> be
> >     > >>>>>> provided uniquely by the consumer client. The broker will
> maintain
> >     > >> a
> >     > >>>>> cache
> >     > >>>>>> with [key:member.name, value:member.id]. A join group
> request with
> >     > >>>>>> member.name set will be treated as `static-membership`
> strategy,
> >     > >> and
> >     > >>>>> will
> >     > >>>>>> reject any join group request without member.name. So this
> >     > >>>> coordination
> >     > >>>>>> change will be differentiated from the `dynamic-membership`
> >     > >> protocol
> >     > >>> we
> >     > >>>>>> currently have.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When handling static join group request:
> >     > >>>>>>
> >     > >>>>>>  1.   The broker will check the membership to see whether
> this is
> >     > >> a
> >     > >>>> new
> >     > >>>>>> member. If new, broker allocate a unique member id, cache
> the
> >     > >> mapping
> >     > >>>> and
> >     > >>>>>> move to rebalance stage.
> >     > >>>>>>  2.   Following 1, if this is an existing member, broker
> will not
> >     > >>>> change
> >     > >>>>>> group state, and return its cached member.id and current
> >     > >> assignment.
> >     > >>>>>> (unless this is leader, we shall trigger rebalance)
> >     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with
> pair
> >     > >>> member
> >     > >>>>>> name and id, I think for join group request it is ok to
> leave
> >     > >> member
> >     > >>> id
> >     > >>>>>> blank as member name is the unique identifier. In commit
> offset
> >     > >>> request
> >     > >>>>> we
> >     > >>>>>> *must* have both.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When handling commit offset request, if enabled with static
> >     > >>> membership,
> >     > >>>>>> each time the commit request must have both member.name and
> >     > >>> member.id
> >     > >>>> to
> >     > >>>>>> be identified as a `certificated member`. If not, this
> means there
> >     > >>> are
> >     > >>>>>> duplicate consumer members with same member name and the
> request
> >     > >> will
> >     > >>>> be
> >     > >>>>>> rejected to guarantee consumption uniqueness.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> When rolling restart/shutting down gracefully, the client
> will
> >     > >> send a
> >     > >>>>>> leave group request (static membership mode). In static
> membership,
> >     > >>> we
> >     > >>>>> will
> >     > >>>>>> also define `change-group-timeout` to hold on rebalance
> provided by
> >     > >>>>> leader.
> >     > >>>>>> So we will wait for all the members to rejoin the group and
> do
> >     > >>> exactly
> >     > >>>>> one
> >     > >>>>>> rebalance since all members are expected to rejoin within
> timeout.
> >     > >> If
> >     > >>>>>> consumer crashes, the join group request from the restarted
> >     > >> consumer
> >     > >>>> will
> >     > >>>>>> be recognized as an existing member and be handled as above
> >     > >> condition
> >     > >>>> 1;
> >     > >>>>>> However, if the consumer takes longer than session timeout
> to
> >     > >> return,
> >     > >>>> we
> >     > >>>>>> shall still trigger rebalance but it could still try to
> catch
> >     > >>>>>> `change-group-timeout`. If it failed to catch second
> timeout, its
> >     > >>>> cached
> >     > >>>>>> state on broker will be garbage collected and trigger a new
> >     > >> rebalance
> >     > >>>>> when
> >     > >>>>>> it finally joins.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> And consider the switch between dynamic to static
> membership.
> >     > >>>>>>
> >     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
> >     > >> membership
> >     > >>>> to
> >     > >>>>>> static and wait for all the current members to restart,
> since their
> >     > >>>>>> membership is still dynamic. Here our assumption is that the
> >     > >> restart
> >     > >>>>>> process shouldn't take a long time, as long restart is
> breaking the
> >     > >>>>>> `rebalance timeout` in whatever membership protocol we are
> using.
> >     > >>>> Before
> >     > >>>>>> restart, all dynamic member join requests will be rejected.
> >     > >>>>>>  2.  Static to dynamic: this is more like a downgrade which
> should
> >     > >>> be
> >     > >>>>>> smooth: just erase the cached mapping, and wait for session
> timeout
> >     > >>> to
> >     > >>>>>> trigger rebalance should be sufficient. (Fallback to current
> >     > >>> behavior)
> >     > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
> >     > >> dynamic
> >     > >>>>>> membership while some keep static membership. This will
> cause the
> >     > >>> group
> >     > >>>>>> rebalance forever without progress because dynamic/static
> states
> >     > >> are
> >     > >>>>>> bouncing each other. This could guarantee that we will not
> make the
> >     > >>>>>> consumer group work in a wrong state by having half static
> and half
> >     > >>>>> dynamic.
> >     > >>>>>>
> >     > >>>>>> To guarantee correctness, we will also push the member
> name/id pair
> >     > >>> to
> >     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and
> upgrade the
> >     > >> API
> >     > >>>>>> version, these details will be further discussed back in
> the KIP.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Are there any concern for this high level proposal? Just
> want to
> >     > >>>>> reiterate
> >     > >>>>>> on the core idea of the KIP: "If the broker recognize this
> consumer
> >     > >>> as
> >     > >>>> an
> >     > >>>>>> existing member, it shouldn't trigger rebalance".
> >     > >>>>>>
> >     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is
> much
> >     > >> more
> >     > >>>>>> robust than previous one!
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> Best,
> >     > >>>>>>
> >     > >>>>>> Boyang
> >     > >>>>>>
> >     > >>>>>> ________________________________
> >     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
> >     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
> >     > >>>>>> To: dev@kafka.apache.org
> >     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer
> rebalances
> >     > >>> by
> >     > >>>>>> specifying member id
> >     > >>>>>>
> >     > >>>>>> Hi,
> >     > >>>>>>
> >     > >>>>>> thanks for the detailed discussion. I learned a lot about
> internals
> >     > >>>> again
> >     > >>>>>> :)
> >     > >>>>>>
> >     > >>>>>> I like the idea or a user config `member.name` and to keep
> `
> >     > >>> member.id`
> >     > >>>>>> internal. Also agree with Guozhang, that reusing `client.id`
> might
> >     > >>> not
> >     > >>>>>> be a good idea.
> >     > >>>>>>
> >     > >>>>>> To clarify the algorithm, each time we generate a new `
> member.id`,
> >     > >>> we
> >     > >>>>>> also need to update the "group membership" information (ie,
> mapping
> >     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id`
> replaces
> >     > >>> the
> >     > >>>>>> old entry in the cache.
> >     > >>>>>>
> >     > >>>>>> I also think, we need to preserve the `member.name ->
> member.id`
> >     > >>>> mapping
> >     > >>>>>> in the `__consumer_offset` topic. The KIP should mention
> this IMHO.
> >     > >>>>>>
> >     > >>>>>> For changing the default value of config
> `leave.group.on.close`. I
> >     > >>>> agree
> >     > >>>>>> with John, that we should not change the default config,
> because it
> >     > >>>>>> would impact all consumer groups with dynamic assignment.
> However,
> >     > >> I
> >     > >>>>>> think we can document, that if static assignment is used
> (ie,
> >     > >>>>>> `member.name` is configured) we never send a
> LeaveGroupRequest
> >     > >>>>>> regardless of the config. Note, that the config is
> internal, so not
> >     > >>>> sure
> >     > >>>>>> how to document this in detail. We should not expose the
> internal
> >     > >>>> config
> >     > >>>>>> in the docs.
> >     > >>>>>>
> >     > >>>>>> About upgrading: why do we need have two rolling bounces
> and encode
> >     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
> >     > >>>>>>
> >     > >>>>>> If we upgrade an existing consumer group from dynamic to
> static, I
> >     > >>>> don't
> >     > >>>>>> see any reason why both should not work together and single
> rolling
> >     > >>>>>> bounce would not be sufficient? If we bounce the first
> consumer and
> >     > >>>>>> switch from dynamic to static, it sends a `member.name`
> and the
> >     > >>> broker
> >     > >>>>>> registers the [member.name, member.id] in the cache. Why
> would
> >     > >> this
> >     > >>>>>> interfere with all other consumer that use dynamic
> assignment?
> >     > >>>>>>
> >     > >>>>>> Also, Guozhang mentioned that for all other request, we
> need to
> >     > >> check
> >     > >>>> if
> >     > >>>>>> the mapping [member.name, member.id] contains the send `
> member.id`
> >     > >>> --
> >     > >>>> I
> >     > >>>>>> don't think this is necessary -- it seems to be sufficient
> to check
> >     > >>> the
> >     > >>>>>> `member.id` from the [member.id, Assignment] mapping as be
> do
> >     > >> today
> >     > >>> --
> >     > >>>>>> thus, checking `member.id` does not require any change
> IMHO.
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>> -Matthias
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
> >     > >>>>>>> @James
> >     > >>>>>>>
> >     > >>>>>>> What you described is true: the transition from dynamic to
> static
> >     > >>>>>>> memberships are not thought through yet. But I do not
> think it is
> >     > >>> an
> >     > >>>>>>> impossible problem: note that we indeed moved the offset
> commit
> >     > >>> from
> >     > >>>> ZK
> >     > >>>>>> to
> >     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to
> first to
> >     > >>>>>>> double-commits on both zk and coordinator, and then do a
> second
> >     > >>> round
> >     > >>>>> to
> >     > >>>>>>> turn the zk off.
> >     > >>>>>>>
> >     > >>>>>>> So just to throw a wild idea here: also following a
> >     > >>>> two-rolling-bounce
> >     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to
> "static"
> >     > >>> while
> >     > >>>>>> keep
> >     > >>>>>>> the registry-id field empty still, in this case, the
> coordinator
> >     > >>>> still
> >     > >>>>>>> follows the logic of "dynamic", accepting the request while
> >     > >>> allowing
> >     > >>>>> the
> >     > >>>>>>> protocol to be set to "static"; after the first rolling
> bounce,
> >     > >> the
> >     > >>>>> group
> >     > >>>>>>> protocol is already "static", then a second rolling bounce
> is
> >     > >>>> triggered
> >     > >>>>>> and
> >     > >>>>>>> this time we set the registry-id.
> >     > >>>>>>>
> >     > >>>>>>>
> >     > >>>>>>> Guozhang
> >     > >>>>>>>
> >     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
> >     > >> wushujames@gmail.com>
> >     > >>>>>> wrote:
> >     > >>>>>>>
> >     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
> >     > >>>>>>>>
> >     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <
> wangguoz@gmail.com
> >     > >>>
> >     > >>>>> wrote:
> >     > >>>>>>>>>
> >     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional
> fields:
> >     > >>>>>>>>>
> >     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
> >     > >>> protocols.
> >     > >>>>>>>>> 1.b) with "static" membership, we also add the
> pre-defined
> >     > >>> member
> >     > >>>>> id.
> >     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
> >     > >>>>>>>>> "group-change-timeout" value.
> >     > >>>>>>>>>
> >     > >>>>>>>>> 2. On the broker side, we enforce only one of the two
> protocols
> >     > >>> for
> >     > >>>>> all
> >     > >>>>>>>>> group members: we accept the protocol on the first joined
> >     > >> member
> >     > >>> of
> >     > >>>>> the
> >     > >>>>>>>>> group, and if later joining members indicate a different
> >     > >>> membership
> >     > >>>>>>>>> protocol, we reject it. If the group-change-timeout
> value was
> >     > >>>>> different
> >     > >>>>>>>> to
> >     > >>>>>>>>> the first joined member, we reject it as well.
> >     > >>>>>>>>
> >     > >>>>>>>>
> >     > >>>>>>>> What will happen if we have an already-deployed
> application that
> >     > >>>> wants
> >     > >>>>>> to
> >     > >>>>>>>> switch to using static membership? Let’s say there are 10
> >     > >>> instances
> >     > >>>> of
> >     > >>>>>> it.
> >     > >>>>>>>> As the instances go through a rolling restart, they will
> switch
> >     > >>> from
> >     > >>>>>>>> dynamic membership (the default?) to static membership.
> As each
> >     > >>> one
> >     > >>>>>> leaves
> >     > >>>>>>>> the group and restarts, they will be rejected from the
> group
> >     > >>>> (because
> >     > >>>>>> the
> >     > >>>>>>>> group is currently using dynamic membership). The group
> will
> >     > >>> shrink
> >     > >>>>> down
> >     > >>>>>>>> until there is 1 node handling all the traffic. After
> that one
> >     > >>>>> restarts,
> >     > >>>>>>>> the group will switch over to static membership.
> >     > >>>>>>>>
> >     > >>>>>>>> Is that right? That means that the transition plan from
> dynamic
> >     > >> to
> >     > >>>>>> static
> >     > >>>>>>>> membership isn’t very smooth.
> >     > >>>>>>>>
> >     > >>>>>>>> I’m not really sure what can be done in this case. This
> reminds
> >     > >> me
> >     > >>>> of
> >     > >>>>>> the
> >     > >>>>>>>> transition plans that were discussed for moving from
> >     > >>> zookeeper-based
> >     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was
> also
> >     > >>> hard,
> >     > >>>>> and
> >     > >>>>>>>> ultimately we decided not to build that.
> >     > >>>>>>>>
> >     > >>>>>>>> -James
> >     > >>>>>>>>
> >     > >>>>>>>>
> >     > >>>>>>>
> >     > >>>>>>>
> >     > >>>>>>
> >     > >>>>>>
> >     > >>>>>
> >     > >>>>
> >     > >>>>
> >     > >>>>
> >     > >>>> --
> >     > >>>> -- Guozhang
> >     > >>>>
> >     > >>>
> >     > >>
> >     > >>
> >     > >>
> >     > >> --
> >     > >> -- Guozhang
> >     > >>
> >     >
> >
> >
>
>

-- 
-Regards,
Mayuresh R. Gharat
(862) 250-7125

Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
I took a quick pass of the proposal. First I would say it's a very brilliant initiative from Konstantine and Confluent folks. To draft up a proposal like this needs deep understanding of the rebalance protocol! I summarized some thoughts here.


Overall the motivations of the two proposals align on that:

  1.  Both believe the invariant resource (belonging to the same process) should be preserved across rebalance.
  2.  Transit failures (K8 thread death) shouldn't trigger resource redistribution. I don't use rebalance here since part one of the cooperative proposal could potentially introduce more rebalances but only on must-move resources.
  3.  Scale up/down and rolling bounce are causing unnecessary resource shuffling that need to be mitigated.


On motivation level, I think both approach could solve/mitigate the above issues. They are just different in design philosophy, or I would say the perspective difference between framework user and algorithm designer.


Two proposals have different focuses. KIP-345 is trying to place more fine-grained control on the broker side to reduce the unnecessary rebalances, while keeping the client logic intact. This is pretty intuitive cause-effect for normal developers who are not very familiar with rebalance protocol. As a developer working with Kafka Streams daily, I'd be happy to see a simplified rebalance protocol and just focus on maintaining the stream/consumer jobs. Too many rebalances raised my concern on the job health. To be concise, static membership has the advantage of reducing mental burden.


Cooperative proposal takes thoughtful approach on client side. We want to have fine-grained control on the join/exit group behaviors and make the current dynamic membership better to address above issues. I do feel our idea crossed on the delayed rebalance when we scale up/down, which could potentially reduce the state shuffling and decouple the behavior from session timeout which is already overloaded.  In this sense, I believe both approaches would serve well in making "reasonable rebalance" happen at the "right timing".


However, based on my understanding, either 345 or cooperative rebalancing is not solving the problem Mike has proposed: could we do a better job at scaling up/down in ideal timing? My initial response was to introduce an admin API which now I feel is sub-optimal, in that the goal of smooth transition is to make sure the newly up hosts are actually "ready". For example:


We have 4 instance reading from 8 topic partitions (= 8 tasks). At some time we would like to scale up to 8 hosts, with the current improvements we could reduce 4 potential rebalances to a single one. But the new hosts are yet unknown to be "ready" if they need to reconstruct the local state. To be actually ready, we need 4 standby tasks running on those empty hosts and leader needs to wait for the signal of "replay/reconstruct complete" to actually involve them into the main consumer group. Otherwise, rebalance just kills our performance since we need to wait indefinite long for task migration.


The scale down is also tricky such that we are not able to define a "true" leave of a member. Rebalance immediately after "true" leaves are most optimal comparing with human intervention. Does this make sense?


My intuition is that cooperative approach which was implemented on the client side could better handle scaling cases than KIP 345, since it involves a lot of algorithmic changes to define "replaying" stage, which I feel would over-complicate broker logic if implemented on coordinator. If we let 345 focus on reducing unnecessary rebalance, and let cooperative approach focus on judging best timing of scale up/down, the two efforts could be aligned. In long term, I feel the more complex improvement of consumer protocol should happen on client side instead of server side which is easier to test and has less global impact for the entire Kafka production cluster.


Thanks again to Konstantine, Matthias and other folks in coming up with this great client proposal. This is great complementation to KIP 345. In a high level, we are not having any collision on the path and both proposals are making sense here. Just need better sync to avoid duplicate effort :)


Best,

Boyang


________________________________
From: Boyang Chen <bc...@outlook.com>
Sent: Wednesday, November 7, 2018 1:57 PM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Thanks Matthias for bringing this awesome proposal up! I shall take a deeper look and make a comparison between the two proposals.


Meanwhile for the scale down specifically for stateful streaming, we could actually introduce a new status called "learner" where the newly up hosts could try to catch up with the assigned task progress first before triggering the rebalance, from which we don't see a sudden dip on the progress. However, it is built on top of the success of KIP-345.


________________________________
From: Matthias J. Sax <ma...@confluent.io>
Sent: Wednesday, November 7, 2018 7:02 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hey,

there was quite a pause on this KIP discussion and in the mean time, a
new design for incremental cooporative rebalance was suggested:

https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies


We should make sure that the proposal and this KIP align to each other.
Thoughts?


-Matthias

On 11/5/18 7:31 PM, Boyang Chen wrote:
> Hey Mike,
>
>
> thanks for the feedback, the two question are very thoughtful!
>
>
>> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I > do not think a bounce of the leader should trigger a rebalance.
>
> For Q1 my intention was to minimize the change within one KIP, since the leader rejoining case could be addressed separately.
>
>
>> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes > to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to > do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on > shrink down. What do you think?
>
> For 2) my understanding is that for scaling down case it is better to be addressed by CLI tool than code logic, since only by human evaluation we could decide whether it is a "right timing" -- the time when all the scaling down consumers are offline -- to kick in rebalance. Unless we introduce another term on coordinator which indicates the target consumer group size, broker will find it hard to decide when to start rebalance. So far I prefer to hold the implementation for that, but agree we could discuss whether we want to introduce admin API in this KIP or a separate one.
>
>
> Thanks again for the proposed ideas!
>
>
> Boyang
>
> ________________________________
> From: Mike Freyberger <mi...@xandr.com>
> Sent: Monday, November 5, 2018 6:13 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
> Boyang,
>
> Thanks for updating the KIP. It's shaping up well. Two things:
>
> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do not think a bounce of the leader should trigger a rebalance.
>
> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on shrink down. What do you think?
>
> Mike
>
> On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
>
>     Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look:
>
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
> cwiki.apache.org
> For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
>
>
>
>
>     KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
>     cwiki.apache.org
>     For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
>
>
>
>
>
>     ________________________________
>     From: Boyang Chen <bc...@outlook.com>
>     Sent: Monday, October 29, 2018 12:34 PM
>     To: dev@kafka.apache.org
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
>     Thanks everyone for the input on this thread! (Sorry it's been a while) I feel that we are very close to the final solution.
>
>
>     Hey Jason and Mike, I have two quick questions on the new features here:
>
>       1.  so our proposal is that until we add a new static member into the group (scale up), we will not trigger rebalance until the "registration timeout"( the member has been offline for too long)? How about leader's rejoin request, I think we should still trigger rebalance when that happens, since the consumer group may have new topics to consume?
>       2.  I'm not very clear on the scale up scenario in static membership here. Should we fallback to dynamic membership while adding/removing hosts (by setting member.name = null), or we still want to add instances with `member.name` so that we eventually expand/shrink the static membership? I personally feel the easier solution is to spin up new members and wait until either the same "registration timeout" or a "scale up timeout" before starting the rebalance. What do you think?
>
>     Meanwhile I will go ahead to make changes to the KIP with our newly discussed items and details. Really excited to see the design has become more solid.
>
>     Best,
>     Boyang
>
>     ________________________________
>     From: Jason Gustafson <ja...@confluent.io>
>     Sent: Saturday, August 25, 2018 6:04 AM
>     To: dev
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
>     Hey Mike,
>
>     Yeah, that's a good point. A long "registration timeout" may not be a great
>     idea. Perhaps in practice you'd set it long enough to be able to detect a
>     failure and provision a new instance. Maybe on the order of 10 minutes is
>     more reasonable.
>
>     In any case, it's probably a good idea to have an administrative way to
>     force deregistration. One option is to extend the DeleteGroups API with a
>     list of members names.
>
>     -Jason
>
>
>
>     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <mf...@appnexus.com>
>     wrote:
>
>     > Jason,
>     >
>     > Regarding step 4 in your proposal which suggests beginning a long timer
>     > (30 minutes) when a static member leaves the group, would there also be the
>     > ability for an admin to force a static membership expiration?
>     >
>     > I’m thinking that during particular types of outages or upgrades users
>     > would want forcefully remove a static member from the group.
>     >
>     > So the user would shut the consumer down normally, which wouldn’t trigger
>     > a rebalance. Then the user could use an admin CLI tool to force remove that
>     > consumer from the group, so the TopicPartitions that were previously owned
>     > by that consumer can be released.
>     >
>     > At a high level, we need consumer groups to gracefully handle intermittent
>     > failures and permanent failures. Currently, the consumer group protocol
>     > handles permanent failures well, but does not handle intermittent failures
>     > well (it creates unnecessary rebalances). I want to make sure the overall
>     > solution here handles both intermittent failures and permanent failures,
>     > rather than sacrificing support for permanent failures in order to provide
>     > support for intermittent failures.
>     >
>     > Mike
>     >
>     > Sent from my iPhone
>     >
>     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io> wrote:
>     > >
>     > > Hey Guozhang,
>     > >
>     > > Responses below:
>     > >
>     > > Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP.
>     > >
>     > >
>     > > I was considering an extension to support pre-initialization of the
>     > static
>     > > members of the group, but I agree we should probably leave this problem
>     > for
>     > > future work.
>     > >
>     > > 1. How this longish static member expiration timeout defined? Is it via a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >
>     > >
>     > > I am not too sure. I tend to lean toward server-side configs because they
>     > > are easier to evolve. If we have to add something to the protocol, then
>     > > we'll be stuck with it forever.
>     > >
>     > > 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >
>     > >
>     > > Yeah, I hope we can ultimately get rid of it, but we may need it for
>     > > compatibility with older brokers. A related question is what should be
>     > the
>     > > behavior of the consumer if `member.name` is provided but the broker
>     > does
>     > > not support it? We could either fail or silently downgrade to dynamic
>     > > membership.
>     > >
>     > > -Jason
>     > >
>     > >
>     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <wa...@gmail.com>
>     > wrote:
>     > >>
>     > >> Hey Jason,
>     > >>
>     > >> I like your idea to simplify the upgrade protocol to allow co-exist of
>     > >> static and dynamic members. Admittedly it may make the coordinator-side
>     > >> logic a bit more complex, but I think it worth doing it.
>     > >>
>     > >> Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP. With that, I think what you've proposed here is a good way
>     > to
>     > >> go for KIP-345 itself.
>     > >>
>     > >> Note there are a few details in your proposal we'd still need to figure
>     > >> out:
>     > >>
>     > >> 1. How this longish static member expiration timeout defined? Is it via
>     > a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >>
>     > >> 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >>
>     > >>
>     > >>
>     > >> Guozhang
>     > >>
>     > >>
>     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <ja...@confluent.io>
>     > >> wrote:
>     > >>
>     > >>> Hey All,
>     > >>>
>     > >>> Nice to see some solid progress on this. It sounds like one of the
>     > >>> complications is allowing static and dynamic registration to coexist.
>     > I'm
>     > >>> wondering if we can do something like the following:
>     > >>>
>     > >>> 1. Statically registered members (those joining the group with a
>     > >> non-null `
>     > >>> member.name`) maintain a session with the coordinator just like
>     > dynamic
>     > >>> members.
>     > >>> 2. If a session is active for a static member when a rebalance begins,
>     > >> then
>     > >>> basically we'll keep the current behavior. The rebalance will await the
>     > >>> static member joining the group.
>     > >>> 3. If a static member does not have an active session, then the
>     > >> coordinator
>     > >>> will not wait for it to join, but will still include it in the
>     > rebalance.
>     > >>> The coordinator will forward the cached subscription information to the
>     > >>> leader and will cache the assignment after the rebalance completes.
>     > (Note
>     > >>> that we still have the generationId to fence offset commits from a
>     > static
>     > >>> zombie if the assignment changes.)
>     > >>> 4. When a static member leaves the group or has its session expire, no
>     > >>> rebalance is triggered. Instead, we can begin a timer to expire the
>     > >> static
>     > >>> registration. This would be a longish timeout (like 30 minutes say).
>     > >>>
>     > >>> So basically static members participate in all rebalances regardless
>     > >>> whether they have an active session. In a given rebalance, some of the
>     > >>> members may be static and some dynamic. The group leader can
>     > >> differentiate
>     > >>> the two based on the presence of the `member.name` (we have to add
>     > this
>     > >> to
>     > >>> the JoinGroupResponse). Generally speaking, we would choose leaders
>     > >>> preferentially from the active members that support the latest
>     > JoinGroup
>     > >>> protocol and are using static membership. If we have to choose a leader
>     > >>> with an old version, however, it would see all members in the group
>     > >> (static
>     > >>> or dynamic) as dynamic members and perform the assignment as usual.
>     > >>>
>     > >>> Would that work?
>     > >>>
>     > >>> -Jason
>     > >>>
>     > >>>
>     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <wa...@gmail.com>
>     > >> wrote:
>     > >>>
>     > >>>> Hello Boyang,
>     > >>>>
>     > >>>> Thanks for the updated proposal, a few questions:
>     > >>>>
>     > >>>> 1. Where will "change-group-timeout" be communicated to the broker?
>     > >> Will
>     > >>>> that be a new field in the JoinGroupRequest, or are we going to
>     > >>> piggy-back
>     > >>>> on the existing session-timeout field (assuming that the original
>     > value
>     > >>>> will not be used anywhere in the static membership any more)?
>     > >>>>
>     > >>>> 2. "However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we shall still trigger rebalance but it could still try to catch
>     > >>>> `change-group-timeout`.": what does this mean? I thought your proposal
>     > >> is
>     > >>>> that for static memberships, the broker will NOT trigger rebalance
>     > even
>     > >>>> after session-timeout has been detected, but only that after
>     > >>>> change-group-timeout
>     > >>>> which is supposed to be longer than session-timeout to be defined?
>     > >>>>
>     > >>>> 3. "A join group request with member.name set will be treated as
>     > >>>> `static-membership` strategy", in this case, how would the switch from
>     > >>>> dynamic to static happen, since whoever changed the member.name to
>     > >>>> not-null
>     > >>>> will be rejected, right?
>     > >>>>
>     > >>>> 4. "just erase the cached mapping, and wait for session timeout to
>     > >>> trigger
>     > >>>> rebalance should be sufficient." this is also a bit unclear to me: who
>     > >>> will
>     > >>>> erase the cached mapping? Since it is on the broker-side I assume that
>     > >>>> broker has to do it. Are you suggesting to use a new request for it?
>     > >>>>
>     > >>>> 5. "Halfway switch": following 3) above, if your proposal is basically
>     > >> to
>     > >>>> let "first join-request wins", and the strategy will stay as is until
>     > >> all
>     > >>>> members are gone, then this will also not happen since whoever used
>     > >>>> different strategy as the first guy who sends join-group request will
>     > >> be
>     > >>>> rejected right?
>     > >>>>
>     > >>>>
>     > >>>> Guozhang
>     > >>>>
>     > >>>>
>     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <jo...@confluent.io>
>     > >> wrote:
>     > >>>>
>     > >>>>> This sounds good to me!
>     > >>>>>
>     > >>>>> Thanks for the time you've spent on it,
>     > >>>>> -John
>     > >>>>>
>     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <bc...@outlook.com>
>     > >>>> wrote:
>     > >>>>>
>     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
>     > >> haven't
>     > >>>> got
>     > >>>>>> time to update this thread. To summarize what we come up so far,
>     > >> here
>     > >>>> is
>     > >>>>> a
>     > >>>>>> draft updated plan:
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Introduce a new config called `member.name` which is supposed to
>     > >> be
>     > >>>>>> provided uniquely by the consumer client. The broker will maintain
>     > >> a
>     > >>>>> cache
>     > >>>>>> with [key:member.name, value:member.id]. A join group request with
>     > >>>>>> member.name set will be treated as `static-membership` strategy,
>     > >> and
>     > >>>>> will
>     > >>>>>> reject any join group request without member.name. So this
>     > >>>> coordination
>     > >>>>>> change will be differentiated from the `dynamic-membership`
>     > >> protocol
>     > >>> we
>     > >>>>>> currently have.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling static join group request:
>     > >>>>>>
>     > >>>>>>  1.   The broker will check the membership to see whether this is
>     > >> a
>     > >>>> new
>     > >>>>>> member. If new, broker allocate a unique member id, cache the
>     > >> mapping
>     > >>>> and
>     > >>>>>> move to rebalance stage.
>     > >>>>>>  2.   Following 1, if this is an existing member, broker will not
>     > >>>> change
>     > >>>>>> group state, and return its cached member.id and current
>     > >> assignment.
>     > >>>>>> (unless this is leader, we shall trigger rebalance)
>     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with pair
>     > >>> member
>     > >>>>>> name and id, I think for join group request it is ok to leave
>     > >> member
>     > >>> id
>     > >>>>>> blank as member name is the unique identifier. In commit offset
>     > >>> request
>     > >>>>> we
>     > >>>>>> *must* have both.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling commit offset request, if enabled with static
>     > >>> membership,
>     > >>>>>> each time the commit request must have both member.name and
>     > >>> member.id
>     > >>>> to
>     > >>>>>> be identified as a `certificated member`. If not, this means there
>     > >>> are
>     > >>>>>> duplicate consumer members with same member name and the request
>     > >> will
>     > >>>> be
>     > >>>>>> rejected to guarantee consumption uniqueness.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When rolling restart/shutting down gracefully, the client will
>     > >> send a
>     > >>>>>> leave group request (static membership mode). In static membership,
>     > >>> we
>     > >>>>> will
>     > >>>>>> also define `change-group-timeout` to hold on rebalance provided by
>     > >>>>> leader.
>     > >>>>>> So we will wait for all the members to rejoin the group and do
>     > >>> exactly
>     > >>>>> one
>     > >>>>>> rebalance since all members are expected to rejoin within timeout.
>     > >> If
>     > >>>>>> consumer crashes, the join group request from the restarted
>     > >> consumer
>     > >>>> will
>     > >>>>>> be recognized as an existing member and be handled as above
>     > >> condition
>     > >>>> 1;
>     > >>>>>> However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we
>     > >>>>>> shall still trigger rebalance but it could still try to catch
>     > >>>>>> `change-group-timeout`. If it failed to catch second timeout, its
>     > >>>> cached
>     > >>>>>> state on broker will be garbage collected and trigger a new
>     > >> rebalance
>     > >>>>> when
>     > >>>>>> it finally joins.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> And consider the switch between dynamic to static membership.
>     > >>>>>>
>     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
>     > >> membership
>     > >>>> to
>     > >>>>>> static and wait for all the current members to restart, since their
>     > >>>>>> membership is still dynamic. Here our assumption is that the
>     > >> restart
>     > >>>>>> process shouldn't take a long time, as long restart is breaking the
>     > >>>>>> `rebalance timeout` in whatever membership protocol we are using.
>     > >>>> Before
>     > >>>>>> restart, all dynamic member join requests will be rejected.
>     > >>>>>>  2.  Static to dynamic: this is more like a downgrade which should
>     > >>> be
>     > >>>>>> smooth: just erase the cached mapping, and wait for session timeout
>     > >>> to
>     > >>>>>> trigger rebalance should be sufficient. (Fallback to current
>     > >>> behavior)
>     > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
>     > >> dynamic
>     > >>>>>> membership while some keep static membership. This will cause the
>     > >>> group
>     > >>>>>> rebalance forever without progress because dynamic/static states
>     > >> are
>     > >>>>>> bouncing each other. This could guarantee that we will not make the
>     > >>>>>> consumer group work in a wrong state by having half static and half
>     > >>>>> dynamic.
>     > >>>>>>
>     > >>>>>> To guarantee correctness, we will also push the member name/id pair
>     > >>> to
>     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and upgrade the
>     > >> API
>     > >>>>>> version, these details will be further discussed back in the KIP.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Are there any concern for this high level proposal? Just want to
>     > >>>>> reiterate
>     > >>>>>> on the core idea of the KIP: "If the broker recognize this consumer
>     > >>> as
>     > >>>> an
>     > >>>>>> existing member, it shouldn't trigger rebalance".
>     > >>>>>>
>     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is much
>     > >> more
>     > >>>>>> robust than previous one!
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Best,
>     > >>>>>>
>     > >>>>>> Boyang
>     > >>>>>>
>     > >>>>>> ________________________________
>     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
>     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
>     > >>>>>> To: dev@kafka.apache.org
>     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
>     > >>> by
>     > >>>>>> specifying member id
>     > >>>>>>
>     > >>>>>> Hi,
>     > >>>>>>
>     > >>>>>> thanks for the detailed discussion. I learned a lot about internals
>     > >>>> again
>     > >>>>>> :)
>     > >>>>>>
>     > >>>>>> I like the idea or a user config `member.name` and to keep `
>     > >>> member.id`
>     > >>>>>> internal. Also agree with Guozhang, that reusing `client.id` might
>     > >>> not
>     > >>>>>> be a good idea.
>     > >>>>>>
>     > >>>>>> To clarify the algorithm, each time we generate a new `member.id`,
>     > >>> we
>     > >>>>>> also need to update the "group membership" information (ie, mapping
>     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id` replaces
>     > >>> the
>     > >>>>>> old entry in the cache.
>     > >>>>>>
>     > >>>>>> I also think, we need to preserve the `member.name -> member.id`
>     > >>>> mapping
>     > >>>>>> in the `__consumer_offset` topic. The KIP should mention this IMHO.
>     > >>>>>>
>     > >>>>>> For changing the default value of config `leave.group.on.close`. I
>     > >>>> agree
>     > >>>>>> with John, that we should not change the default config, because it
>     > >>>>>> would impact all consumer groups with dynamic assignment. However,
>     > >> I
>     > >>>>>> think we can document, that if static assignment is used (ie,
>     > >>>>>> `member.name` is configured) we never send a LeaveGroupRequest
>     > >>>>>> regardless of the config. Note, that the config is internal, so not
>     > >>>> sure
>     > >>>>>> how to document this in detail. We should not expose the internal
>     > >>>> config
>     > >>>>>> in the docs.
>     > >>>>>>
>     > >>>>>> About upgrading: why do we need have two rolling bounces and encode
>     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
>     > >>>>>>
>     > >>>>>> If we upgrade an existing consumer group from dynamic to static, I
>     > >>>> don't
>     > >>>>>> see any reason why both should not work together and single rolling
>     > >>>>>> bounce would not be sufficient? If we bounce the first consumer and
>     > >>>>>> switch from dynamic to static, it sends a `member.name` and the
>     > >>> broker
>     > >>>>>> registers the [member.name, member.id] in the cache. Why would
>     > >> this
>     > >>>>>> interfere with all other consumer that use dynamic assignment?
>     > >>>>>>
>     > >>>>>> Also, Guozhang mentioned that for all other request, we need to
>     > >> check
>     > >>>> if
>     > >>>>>> the mapping [member.name, member.id] contains the send `member.id`
>     > >>> --
>     > >>>> I
>     > >>>>>> don't think this is necessary -- it seems to be sufficient to check
>     > >>> the
>     > >>>>>> `member.id` from the [member.id, Assignment] mapping as be do
>     > >> today
>     > >>> --
>     > >>>>>> thus, checking `member.id` does not require any change IMHO.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> -Matthias
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
>     > >>>>>>> @James
>     > >>>>>>>
>     > >>>>>>> What you described is true: the transition from dynamic to static
>     > >>>>>>> memberships are not thought through yet. But I do not think it is
>     > >>> an
>     > >>>>>>> impossible problem: note that we indeed moved the offset commit
>     > >>> from
>     > >>>> ZK
>     > >>>>>> to
>     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to first to
>     > >>>>>>> double-commits on both zk and coordinator, and then do a second
>     > >>> round
>     > >>>>> to
>     > >>>>>>> turn the zk off.
>     > >>>>>>>
>     > >>>>>>> So just to throw a wild idea here: also following a
>     > >>>> two-rolling-bounce
>     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to "static"
>     > >>> while
>     > >>>>>> keep
>     > >>>>>>> the registry-id field empty still, in this case, the coordinator
>     > >>>> still
>     > >>>>>>> follows the logic of "dynamic", accepting the request while
>     > >>> allowing
>     > >>>>> the
>     > >>>>>>> protocol to be set to "static"; after the first rolling bounce,
>     > >> the
>     > >>>>> group
>     > >>>>>>> protocol is already "static", then a second rolling bounce is
>     > >>>> triggered
>     > >>>>>> and
>     > >>>>>>> this time we set the registry-id.
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> Guozhang
>     > >>>>>>>
>     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
>     > >> wushujames@gmail.com>
>     > >>>>>> wrote:
>     > >>>>>>>
>     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
>     > >>>>>>>>
>     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <wangguoz@gmail.com
>     > >>>
>     > >>>>> wrote:
>     > >>>>>>>>>
>     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional fields:
>     > >>>>>>>>>
>     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
>     > >>> protocols.
>     > >>>>>>>>> 1.b) with "static" membership, we also add the pre-defined
>     > >>> member
>     > >>>>> id.
>     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
>     > >>>>>>>>> "group-change-timeout" value.
>     > >>>>>>>>>
>     > >>>>>>>>> 2. On the broker side, we enforce only one of the two protocols
>     > >>> for
>     > >>>>> all
>     > >>>>>>>>> group members: we accept the protocol on the first joined
>     > >> member
>     > >>> of
>     > >>>>> the
>     > >>>>>>>>> group, and if later joining members indicate a different
>     > >>> membership
>     > >>>>>>>>> protocol, we reject it. If the group-change-timeout value was
>     > >>>>> different
>     > >>>>>>>> to
>     > >>>>>>>>> the first joined member, we reject it as well.
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> What will happen if we have an already-deployed application that
>     > >>>> wants
>     > >>>>>> to
>     > >>>>>>>> switch to using static membership? Let’s say there are 10
>     > >>> instances
>     > >>>> of
>     > >>>>>> it.
>     > >>>>>>>> As the instances go through a rolling restart, they will switch
>     > >>> from
>     > >>>>>>>> dynamic membership (the default?) to static membership. As each
>     > >>> one
>     > >>>>>> leaves
>     > >>>>>>>> the group and restarts, they will be rejected from the group
>     > >>>> (because
>     > >>>>>> the
>     > >>>>>>>> group is currently using dynamic membership). The group will
>     > >>> shrink
>     > >>>>> down
>     > >>>>>>>> until there is 1 node handling all the traffic. After that one
>     > >>>>> restarts,
>     > >>>>>>>> the group will switch over to static membership.
>     > >>>>>>>>
>     > >>>>>>>> Is that right? That means that the transition plan from dynamic
>     > >> to
>     > >>>>>> static
>     > >>>>>>>> membership isn’t very smooth.
>     > >>>>>>>>
>     > >>>>>>>> I’m not really sure what can be done in this case. This reminds
>     > >> me
>     > >>>> of
>     > >>>>>> the
>     > >>>>>>>> transition plans that were discussed for moving from
>     > >>> zookeeper-based
>     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was also
>     > >>> hard,
>     > >>>>> and
>     > >>>>>>>> ultimately we decided not to build that.
>     > >>>>>>>>
>     > >>>>>>>> -James
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>> --
>     > >>>> -- Guozhang
>     > >>>>
>     > >>>
>     > >>
>     > >>
>     > >>
>     > >> --
>     > >> -- Guozhang
>     > >>
>     >
>
>


Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Thanks Matthias for bringing this awesome proposal up! I shall take a deeper look and make a comparison between the two proposals.


Meanwhile for the scale down specifically for stateful streaming, we could actually introduce a new status called "learner" where the newly up hosts could try to catch up with the assigned task progress first before triggering the rebalance, from which we don't see a sudden dip on the progress. However, it is built on top of the success of KIP-345.


________________________________
From: Matthias J. Sax <ma...@confluent.io>
Sent: Wednesday, November 7, 2018 7:02 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Hey,

there was quite a pause on this KIP discussion and in the mean time, a
new design for incremental cooporative rebalance was suggested:

https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies


We should make sure that the proposal and this KIP align to each other.
Thoughts?


-Matthias

On 11/5/18 7:31 PM, Boyang Chen wrote:
> Hey Mike,
>
>
> thanks for the feedback, the two question are very thoughtful!
>
>
>> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I > do not think a bounce of the leader should trigger a rebalance.
>
> For Q1 my intention was to minimize the change within one KIP, since the leader rejoining case could be addressed separately.
>
>
>> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes > to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to > do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on > shrink down. What do you think?
>
> For 2) my understanding is that for scaling down case it is better to be addressed by CLI tool than code logic, since only by human evaluation we could decide whether it is a "right timing" -- the time when all the scaling down consumers are offline -- to kick in rebalance. Unless we introduce another term on coordinator which indicates the target consumer group size, broker will find it hard to decide when to start rebalance. So far I prefer to hold the implementation for that, but agree we could discuss whether we want to introduce admin API in this KIP or a separate one.
>
>
> Thanks again for the proposed ideas!
>
>
> Boyang
>
> ________________________________
> From: Mike Freyberger <mi...@xandr.com>
> Sent: Monday, November 5, 2018 6:13 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
> Boyang,
>
> Thanks for updating the KIP. It's shaping up well. Two things:
>
> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do not think a bounce of the leader should trigger a rebalance.
>
> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on shrink down. What do you think?
>
> Mike
>
> On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
>
>     Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look:
>
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
> cwiki.apache.org
> For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
>
>
>
>
>     KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
>     cwiki.apache.org
>     For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
>
>
>
>
>
>     ________________________________
>     From: Boyang Chen <bc...@outlook.com>
>     Sent: Monday, October 29, 2018 12:34 PM
>     To: dev@kafka.apache.org
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
>     Thanks everyone for the input on this thread! (Sorry it's been a while) I feel that we are very close to the final solution.
>
>
>     Hey Jason and Mike, I have two quick questions on the new features here:
>
>       1.  so our proposal is that until we add a new static member into the group (scale up), we will not trigger rebalance until the "registration timeout"( the member has been offline for too long)? How about leader's rejoin request, I think we should still trigger rebalance when that happens, since the consumer group may have new topics to consume?
>       2.  I'm not very clear on the scale up scenario in static membership here. Should we fallback to dynamic membership while adding/removing hosts (by setting member.name = null), or we still want to add instances with `member.name` so that we eventually expand/shrink the static membership? I personally feel the easier solution is to spin up new members and wait until either the same "registration timeout" or a "scale up timeout" before starting the rebalance. What do you think?
>
>     Meanwhile I will go ahead to make changes to the KIP with our newly discussed items and details. Really excited to see the design has become more solid.
>
>     Best,
>     Boyang
>
>     ________________________________
>     From: Jason Gustafson <ja...@confluent.io>
>     Sent: Saturday, August 25, 2018 6:04 AM
>     To: dev
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
>
>     Hey Mike,
>
>     Yeah, that's a good point. A long "registration timeout" may not be a great
>     idea. Perhaps in practice you'd set it long enough to be able to detect a
>     failure and provision a new instance. Maybe on the order of 10 minutes is
>     more reasonable.
>
>     In any case, it's probably a good idea to have an administrative way to
>     force deregistration. One option is to extend the DeleteGroups API with a
>     list of members names.
>
>     -Jason
>
>
>
>     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <mf...@appnexus.com>
>     wrote:
>
>     > Jason,
>     >
>     > Regarding step 4 in your proposal which suggests beginning a long timer
>     > (30 minutes) when a static member leaves the group, would there also be the
>     > ability for an admin to force a static membership expiration?
>     >
>     > I’m thinking that during particular types of outages or upgrades users
>     > would want forcefully remove a static member from the group.
>     >
>     > So the user would shut the consumer down normally, which wouldn’t trigger
>     > a rebalance. Then the user could use an admin CLI tool to force remove that
>     > consumer from the group, so the TopicPartitions that were previously owned
>     > by that consumer can be released.
>     >
>     > At a high level, we need consumer groups to gracefully handle intermittent
>     > failures and permanent failures. Currently, the consumer group protocol
>     > handles permanent failures well, but does not handle intermittent failures
>     > well (it creates unnecessary rebalances). I want to make sure the overall
>     > solution here handles both intermittent failures and permanent failures,
>     > rather than sacrificing support for permanent failures in order to provide
>     > support for intermittent failures.
>     >
>     > Mike
>     >
>     > Sent from my iPhone
>     >
>     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io> wrote:
>     > >
>     > > Hey Guozhang,
>     > >
>     > > Responses below:
>     > >
>     > > Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP.
>     > >
>     > >
>     > > I was considering an extension to support pre-initialization of the
>     > static
>     > > members of the group, but I agree we should probably leave this problem
>     > for
>     > > future work.
>     > >
>     > > 1. How this longish static member expiration timeout defined? Is it via a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >
>     > >
>     > > I am not too sure. I tend to lean toward server-side configs because they
>     > > are easier to evolve. If we have to add something to the protocol, then
>     > > we'll be stuck with it forever.
>     > >
>     > > 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >
>     > >
>     > > Yeah, I hope we can ultimately get rid of it, but we may need it for
>     > > compatibility with older brokers. A related question is what should be
>     > the
>     > > behavior of the consumer if `member.name` is provided but the broker
>     > does
>     > > not support it? We could either fail or silently downgrade to dynamic
>     > > membership.
>     > >
>     > > -Jason
>     > >
>     > >
>     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <wa...@gmail.com>
>     > wrote:
>     > >>
>     > >> Hey Jason,
>     > >>
>     > >> I like your idea to simplify the upgrade protocol to allow co-exist of
>     > >> static and dynamic members. Admittedly it may make the coordinator-side
>     > >> logic a bit more complex, but I think it worth doing it.
>     > >>
>     > >> Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP. With that, I think what you've proposed here is a good way
>     > to
>     > >> go for KIP-345 itself.
>     > >>
>     > >> Note there are a few details in your proposal we'd still need to figure
>     > >> out:
>     > >>
>     > >> 1. How this longish static member expiration timeout defined? Is it via
>     > a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >>
>     > >> 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >>
>     > >>
>     > >>
>     > >> Guozhang
>     > >>
>     > >>
>     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <ja...@confluent.io>
>     > >> wrote:
>     > >>
>     > >>> Hey All,
>     > >>>
>     > >>> Nice to see some solid progress on this. It sounds like one of the
>     > >>> complications is allowing static and dynamic registration to coexist.
>     > I'm
>     > >>> wondering if we can do something like the following:
>     > >>>
>     > >>> 1. Statically registered members (those joining the group with a
>     > >> non-null `
>     > >>> member.name`) maintain a session with the coordinator just like
>     > dynamic
>     > >>> members.
>     > >>> 2. If a session is active for a static member when a rebalance begins,
>     > >> then
>     > >>> basically we'll keep the current behavior. The rebalance will await the
>     > >>> static member joining the group.
>     > >>> 3. If a static member does not have an active session, then the
>     > >> coordinator
>     > >>> will not wait for it to join, but will still include it in the
>     > rebalance.
>     > >>> The coordinator will forward the cached subscription information to the
>     > >>> leader and will cache the assignment after the rebalance completes.
>     > (Note
>     > >>> that we still have the generationId to fence offset commits from a
>     > static
>     > >>> zombie if the assignment changes.)
>     > >>> 4. When a static member leaves the group or has its session expire, no
>     > >>> rebalance is triggered. Instead, we can begin a timer to expire the
>     > >> static
>     > >>> registration. This would be a longish timeout (like 30 minutes say).
>     > >>>
>     > >>> So basically static members participate in all rebalances regardless
>     > >>> whether they have an active session. In a given rebalance, some of the
>     > >>> members may be static and some dynamic. The group leader can
>     > >> differentiate
>     > >>> the two based on the presence of the `member.name` (we have to add
>     > this
>     > >> to
>     > >>> the JoinGroupResponse). Generally speaking, we would choose leaders
>     > >>> preferentially from the active members that support the latest
>     > JoinGroup
>     > >>> protocol and are using static membership. If we have to choose a leader
>     > >>> with an old version, however, it would see all members in the group
>     > >> (static
>     > >>> or dynamic) as dynamic members and perform the assignment as usual.
>     > >>>
>     > >>> Would that work?
>     > >>>
>     > >>> -Jason
>     > >>>
>     > >>>
>     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <wa...@gmail.com>
>     > >> wrote:
>     > >>>
>     > >>>> Hello Boyang,
>     > >>>>
>     > >>>> Thanks for the updated proposal, a few questions:
>     > >>>>
>     > >>>> 1. Where will "change-group-timeout" be communicated to the broker?
>     > >> Will
>     > >>>> that be a new field in the JoinGroupRequest, or are we going to
>     > >>> piggy-back
>     > >>>> on the existing session-timeout field (assuming that the original
>     > value
>     > >>>> will not be used anywhere in the static membership any more)?
>     > >>>>
>     > >>>> 2. "However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we shall still trigger rebalance but it could still try to catch
>     > >>>> `change-group-timeout`.": what does this mean? I thought your proposal
>     > >> is
>     > >>>> that for static memberships, the broker will NOT trigger rebalance
>     > even
>     > >>>> after session-timeout has been detected, but only that after
>     > >>>> change-group-timeout
>     > >>>> which is supposed to be longer than session-timeout to be defined?
>     > >>>>
>     > >>>> 3. "A join group request with member.name set will be treated as
>     > >>>> `static-membership` strategy", in this case, how would the switch from
>     > >>>> dynamic to static happen, since whoever changed the member.name to
>     > >>>> not-null
>     > >>>> will be rejected, right?
>     > >>>>
>     > >>>> 4. "just erase the cached mapping, and wait for session timeout to
>     > >>> trigger
>     > >>>> rebalance should be sufficient." this is also a bit unclear to me: who
>     > >>> will
>     > >>>> erase the cached mapping? Since it is on the broker-side I assume that
>     > >>>> broker has to do it. Are you suggesting to use a new request for it?
>     > >>>>
>     > >>>> 5. "Halfway switch": following 3) above, if your proposal is basically
>     > >> to
>     > >>>> let "first join-request wins", and the strategy will stay as is until
>     > >> all
>     > >>>> members are gone, then this will also not happen since whoever used
>     > >>>> different strategy as the first guy who sends join-group request will
>     > >> be
>     > >>>> rejected right?
>     > >>>>
>     > >>>>
>     > >>>> Guozhang
>     > >>>>
>     > >>>>
>     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <jo...@confluent.io>
>     > >> wrote:
>     > >>>>
>     > >>>>> This sounds good to me!
>     > >>>>>
>     > >>>>> Thanks for the time you've spent on it,
>     > >>>>> -John
>     > >>>>>
>     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <bc...@outlook.com>
>     > >>>> wrote:
>     > >>>>>
>     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
>     > >> haven't
>     > >>>> got
>     > >>>>>> time to update this thread. To summarize what we come up so far,
>     > >> here
>     > >>>> is
>     > >>>>> a
>     > >>>>>> draft updated plan:
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Introduce a new config called `member.name` which is supposed to
>     > >> be
>     > >>>>>> provided uniquely by the consumer client. The broker will maintain
>     > >> a
>     > >>>>> cache
>     > >>>>>> with [key:member.name, value:member.id]. A join group request with
>     > >>>>>> member.name set will be treated as `static-membership` strategy,
>     > >> and
>     > >>>>> will
>     > >>>>>> reject any join group request without member.name. So this
>     > >>>> coordination
>     > >>>>>> change will be differentiated from the `dynamic-membership`
>     > >> protocol
>     > >>> we
>     > >>>>>> currently have.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling static join group request:
>     > >>>>>>
>     > >>>>>>  1.   The broker will check the membership to see whether this is
>     > >> a
>     > >>>> new
>     > >>>>>> member. If new, broker allocate a unique member id, cache the
>     > >> mapping
>     > >>>> and
>     > >>>>>> move to rebalance stage.
>     > >>>>>>  2.   Following 1, if this is an existing member, broker will not
>     > >>>> change
>     > >>>>>> group state, and return its cached member.id and current
>     > >> assignment.
>     > >>>>>> (unless this is leader, we shall trigger rebalance)
>     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with pair
>     > >>> member
>     > >>>>>> name and id, I think for join group request it is ok to leave
>     > >> member
>     > >>> id
>     > >>>>>> blank as member name is the unique identifier. In commit offset
>     > >>> request
>     > >>>>> we
>     > >>>>>> *must* have both.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling commit offset request, if enabled with static
>     > >>> membership,
>     > >>>>>> each time the commit request must have both member.name and
>     > >>> member.id
>     > >>>> to
>     > >>>>>> be identified as a `certificated member`. If not, this means there
>     > >>> are
>     > >>>>>> duplicate consumer members with same member name and the request
>     > >> will
>     > >>>> be
>     > >>>>>> rejected to guarantee consumption uniqueness.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When rolling restart/shutting down gracefully, the client will
>     > >> send a
>     > >>>>>> leave group request (static membership mode). In static membership,
>     > >>> we
>     > >>>>> will
>     > >>>>>> also define `change-group-timeout` to hold on rebalance provided by
>     > >>>>> leader.
>     > >>>>>> So we will wait for all the members to rejoin the group and do
>     > >>> exactly
>     > >>>>> one
>     > >>>>>> rebalance since all members are expected to rejoin within timeout.
>     > >> If
>     > >>>>>> consumer crashes, the join group request from the restarted
>     > >> consumer
>     > >>>> will
>     > >>>>>> be recognized as an existing member and be handled as above
>     > >> condition
>     > >>>> 1;
>     > >>>>>> However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we
>     > >>>>>> shall still trigger rebalance but it could still try to catch
>     > >>>>>> `change-group-timeout`. If it failed to catch second timeout, its
>     > >>>> cached
>     > >>>>>> state on broker will be garbage collected and trigger a new
>     > >> rebalance
>     > >>>>> when
>     > >>>>>> it finally joins.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> And consider the switch between dynamic to static membership.
>     > >>>>>>
>     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
>     > >> membership
>     > >>>> to
>     > >>>>>> static and wait for all the current members to restart, since their
>     > >>>>>> membership is still dynamic. Here our assumption is that the
>     > >> restart
>     > >>>>>> process shouldn't take a long time, as long restart is breaking the
>     > >>>>>> `rebalance timeout` in whatever membership protocol we are using.
>     > >>>> Before
>     > >>>>>> restart, all dynamic member join requests will be rejected.
>     > >>>>>>  2.  Static to dynamic: this is more like a downgrade which should
>     > >>> be
>     > >>>>>> smooth: just erase the cached mapping, and wait for session timeout
>     > >>> to
>     > >>>>>> trigger rebalance should be sufficient. (Fallback to current
>     > >>> behavior)
>     > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
>     > >> dynamic
>     > >>>>>> membership while some keep static membership. This will cause the
>     > >>> group
>     > >>>>>> rebalance forever without progress because dynamic/static states
>     > >> are
>     > >>>>>> bouncing each other. This could guarantee that we will not make the
>     > >>>>>> consumer group work in a wrong state by having half static and half
>     > >>>>> dynamic.
>     > >>>>>>
>     > >>>>>> To guarantee correctness, we will also push the member name/id pair
>     > >>> to
>     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and upgrade the
>     > >> API
>     > >>>>>> version, these details will be further discussed back in the KIP.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Are there any concern for this high level proposal? Just want to
>     > >>>>> reiterate
>     > >>>>>> on the core idea of the KIP: "If the broker recognize this consumer
>     > >>> as
>     > >>>> an
>     > >>>>>> existing member, it shouldn't trigger rebalance".
>     > >>>>>>
>     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is much
>     > >> more
>     > >>>>>> robust than previous one!
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Best,
>     > >>>>>>
>     > >>>>>> Boyang
>     > >>>>>>
>     > >>>>>> ________________________________
>     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
>     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
>     > >>>>>> To: dev@kafka.apache.org
>     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
>     > >>> by
>     > >>>>>> specifying member id
>     > >>>>>>
>     > >>>>>> Hi,
>     > >>>>>>
>     > >>>>>> thanks for the detailed discussion. I learned a lot about internals
>     > >>>> again
>     > >>>>>> :)
>     > >>>>>>
>     > >>>>>> I like the idea or a user config `member.name` and to keep `
>     > >>> member.id`
>     > >>>>>> internal. Also agree with Guozhang, that reusing `client.id` might
>     > >>> not
>     > >>>>>> be a good idea.
>     > >>>>>>
>     > >>>>>> To clarify the algorithm, each time we generate a new `member.id`,
>     > >>> we
>     > >>>>>> also need to update the "group membership" information (ie, mapping
>     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id` replaces
>     > >>> the
>     > >>>>>> old entry in the cache.
>     > >>>>>>
>     > >>>>>> I also think, we need to preserve the `member.name -> member.id`
>     > >>>> mapping
>     > >>>>>> in the `__consumer_offset` topic. The KIP should mention this IMHO.
>     > >>>>>>
>     > >>>>>> For changing the default value of config `leave.group.on.close`. I
>     > >>>> agree
>     > >>>>>> with John, that we should not change the default config, because it
>     > >>>>>> would impact all consumer groups with dynamic assignment. However,
>     > >> I
>     > >>>>>> think we can document, that if static assignment is used (ie,
>     > >>>>>> `member.name` is configured) we never send a LeaveGroupRequest
>     > >>>>>> regardless of the config. Note, that the config is internal, so not
>     > >>>> sure
>     > >>>>>> how to document this in detail. We should not expose the internal
>     > >>>> config
>     > >>>>>> in the docs.
>     > >>>>>>
>     > >>>>>> About upgrading: why do we need have two rolling bounces and encode
>     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
>     > >>>>>>
>     > >>>>>> If we upgrade an existing consumer group from dynamic to static, I
>     > >>>> don't
>     > >>>>>> see any reason why both should not work together and single rolling
>     > >>>>>> bounce would not be sufficient? If we bounce the first consumer and
>     > >>>>>> switch from dynamic to static, it sends a `member.name` and the
>     > >>> broker
>     > >>>>>> registers the [member.name, member.id] in the cache. Why would
>     > >> this
>     > >>>>>> interfere with all other consumer that use dynamic assignment?
>     > >>>>>>
>     > >>>>>> Also, Guozhang mentioned that for all other request, we need to
>     > >> check
>     > >>>> if
>     > >>>>>> the mapping [member.name, member.id] contains the send `member.id`
>     > >>> --
>     > >>>> I
>     > >>>>>> don't think this is necessary -- it seems to be sufficient to check
>     > >>> the
>     > >>>>>> `member.id` from the [member.id, Assignment] mapping as be do
>     > >> today
>     > >>> --
>     > >>>>>> thus, checking `member.id` does not require any change IMHO.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> -Matthias
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
>     > >>>>>>> @James
>     > >>>>>>>
>     > >>>>>>> What you described is true: the transition from dynamic to static
>     > >>>>>>> memberships are not thought through yet. But I do not think it is
>     > >>> an
>     > >>>>>>> impossible problem: note that we indeed moved the offset commit
>     > >>> from
>     > >>>> ZK
>     > >>>>>> to
>     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to first to
>     > >>>>>>> double-commits on both zk and coordinator, and then do a second
>     > >>> round
>     > >>>>> to
>     > >>>>>>> turn the zk off.
>     > >>>>>>>
>     > >>>>>>> So just to throw a wild idea here: also following a
>     > >>>> two-rolling-bounce
>     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to "static"
>     > >>> while
>     > >>>>>> keep
>     > >>>>>>> the registry-id field empty still, in this case, the coordinator
>     > >>>> still
>     > >>>>>>> follows the logic of "dynamic", accepting the request while
>     > >>> allowing
>     > >>>>> the
>     > >>>>>>> protocol to be set to "static"; after the first rolling bounce,
>     > >> the
>     > >>>>> group
>     > >>>>>>> protocol is already "static", then a second rolling bounce is
>     > >>>> triggered
>     > >>>>>> and
>     > >>>>>>> this time we set the registry-id.
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> Guozhang
>     > >>>>>>>
>     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
>     > >> wushujames@gmail.com>
>     > >>>>>> wrote:
>     > >>>>>>>
>     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
>     > >>>>>>>>
>     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <wangguoz@gmail.com
>     > >>>
>     > >>>>> wrote:
>     > >>>>>>>>>
>     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional fields:
>     > >>>>>>>>>
>     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
>     > >>> protocols.
>     > >>>>>>>>> 1.b) with "static" membership, we also add the pre-defined
>     > >>> member
>     > >>>>> id.
>     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
>     > >>>>>>>>> "group-change-timeout" value.
>     > >>>>>>>>>
>     > >>>>>>>>> 2. On the broker side, we enforce only one of the two protocols
>     > >>> for
>     > >>>>> all
>     > >>>>>>>>> group members: we accept the protocol on the first joined
>     > >> member
>     > >>> of
>     > >>>>> the
>     > >>>>>>>>> group, and if later joining members indicate a different
>     > >>> membership
>     > >>>>>>>>> protocol, we reject it. If the group-change-timeout value was
>     > >>>>> different
>     > >>>>>>>> to
>     > >>>>>>>>> the first joined member, we reject it as well.
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> What will happen if we have an already-deployed application that
>     > >>>> wants
>     > >>>>>> to
>     > >>>>>>>> switch to using static membership? Let’s say there are 10
>     > >>> instances
>     > >>>> of
>     > >>>>>> it.
>     > >>>>>>>> As the instances go through a rolling restart, they will switch
>     > >>> from
>     > >>>>>>>> dynamic membership (the default?) to static membership. As each
>     > >>> one
>     > >>>>>> leaves
>     > >>>>>>>> the group and restarts, they will be rejected from the group
>     > >>>> (because
>     > >>>>>> the
>     > >>>>>>>> group is currently using dynamic membership). The group will
>     > >>> shrink
>     > >>>>> down
>     > >>>>>>>> until there is 1 node handling all the traffic. After that one
>     > >>>>> restarts,
>     > >>>>>>>> the group will switch over to static membership.
>     > >>>>>>>>
>     > >>>>>>>> Is that right? That means that the transition plan from dynamic
>     > >> to
>     > >>>>>> static
>     > >>>>>>>> membership isn’t very smooth.
>     > >>>>>>>>
>     > >>>>>>>> I’m not really sure what can be done in this case. This reminds
>     > >> me
>     > >>>> of
>     > >>>>>> the
>     > >>>>>>>> transition plans that were discussed for moving from
>     > >>> zookeeper-based
>     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was also
>     > >>> hard,
>     > >>>>> and
>     > >>>>>>>> ultimately we decided not to build that.
>     > >>>>>>>>
>     > >>>>>>>> -James
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>> --
>     > >>>> -- Guozhang
>     > >>>>
>     > >>>
>     > >>
>     > >>
>     > >>
>     > >> --
>     > >> -- Guozhang
>     > >>
>     >
>
>


Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by "Matthias J. Sax" <ma...@confluent.io>.
Hey,

there was quite a pause on this KIP discussion and in the mean time, a
new design for incremental cooporative rebalance was suggested:

https://cwiki.apache.org/confluence/display/KAFKA/Incremental+Cooperative+Rebalancing%3A+Support+and+Policies


We should make sure that the proposal and this KIP align to each other.
Thoughts?


-Matthias

On 11/5/18 7:31 PM, Boyang Chen wrote:
> Hey Mike,
> 
> 
> thanks for the feedback, the two question are very thoughtful!
> 
> 
>> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I > do not think a bounce of the leader should trigger a rebalance.
> 
> For Q1 my intention was to minimize the change within one KIP, since the leader rejoining case could be addressed separately.
> 
> 
>> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes > to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to > do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on > shrink down. What do you think?
> 
> For 2) my understanding is that for scaling down case it is better to be addressed by CLI tool than code logic, since only by human evaluation we could decide whether it is a "right timing" -- the time when all the scaling down consumers are offline -- to kick in rebalance. Unless we introduce another term on coordinator which indicates the target consumer group size, broker will find it hard to decide when to start rebalance. So far I prefer to hold the implementation for that, but agree we could discuss whether we want to introduce admin API in this KIP or a separate one.
> 
> 
> Thanks again for the proposed ideas!
> 
> 
> Boyang
> 
> ________________________________
> From: Mike Freyberger <mi...@xandr.com>
> Sent: Monday, November 5, 2018 6:13 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
> 
> Boyang,
> 
> Thanks for updating the KIP. It's shaping up well. Two things:
> 
> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do not think a bounce of the leader should trigger a rebalance.
> 
> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on shrink down. What do you think?
> 
> Mike
> 
> On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:
> 
>     Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look:
> 
>     https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
> KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
> cwiki.apache.org
> For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
> 
> 
> 
> 
>     KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
>     cwiki.apache.org
>     For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...
> 
> 
> 
> 
> 
>     ________________________________
>     From: Boyang Chen <bc...@outlook.com>
>     Sent: Monday, October 29, 2018 12:34 PM
>     To: dev@kafka.apache.org
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
> 
>     Thanks everyone for the input on this thread! (Sorry it's been a while) I feel that we are very close to the final solution.
> 
> 
>     Hey Jason and Mike, I have two quick questions on the new features here:
> 
>       1.  so our proposal is that until we add a new static member into the group (scale up), we will not trigger rebalance until the "registration timeout"( the member has been offline for too long)? How about leader's rejoin request, I think we should still trigger rebalance when that happens, since the consumer group may have new topics to consume?
>       2.  I'm not very clear on the scale up scenario in static membership here. Should we fallback to dynamic membership while adding/removing hosts (by setting member.name = null), or we still want to add instances with `member.name` so that we eventually expand/shrink the static membership? I personally feel the easier solution is to spin up new members and wait until either the same "registration timeout" or a "scale up timeout" before starting the rebalance. What do you think?
> 
>     Meanwhile I will go ahead to make changes to the KIP with our newly discussed items and details. Really excited to see the design has become more solid.
> 
>     Best,
>     Boyang
> 
>     ________________________________
>     From: Jason Gustafson <ja...@confluent.io>
>     Sent: Saturday, August 25, 2018 6:04 AM
>     To: dev
>     Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id
> 
>     Hey Mike,
> 
>     Yeah, that's a good point. A long "registration timeout" may not be a great
>     idea. Perhaps in practice you'd set it long enough to be able to detect a
>     failure and provision a new instance. Maybe on the order of 10 minutes is
>     more reasonable.
> 
>     In any case, it's probably a good idea to have an administrative way to
>     force deregistration. One option is to extend the DeleteGroups API with a
>     list of members names.
> 
>     -Jason
> 
> 
> 
>     On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <mf...@appnexus.com>
>     wrote:
> 
>     > Jason,
>     >
>     > Regarding step 4 in your proposal which suggests beginning a long timer
>     > (30 minutes) when a static member leaves the group, would there also be the
>     > ability for an admin to force a static membership expiration?
>     >
>     > I’m thinking that during particular types of outages or upgrades users
>     > would want forcefully remove a static member from the group.
>     >
>     > So the user would shut the consumer down normally, which wouldn’t trigger
>     > a rebalance. Then the user could use an admin CLI tool to force remove that
>     > consumer from the group, so the TopicPartitions that were previously owned
>     > by that consumer can be released.
>     >
>     > At a high level, we need consumer groups to gracefully handle intermittent
>     > failures and permanent failures. Currently, the consumer group protocol
>     > handles permanent failures well, but does not handle intermittent failures
>     > well (it creates unnecessary rebalances). I want to make sure the overall
>     > solution here handles both intermittent failures and permanent failures,
>     > rather than sacrificing support for permanent failures in order to provide
>     > support for intermittent failures.
>     >
>     > Mike
>     >
>     > Sent from my iPhone
>     >
>     > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io> wrote:
>     > >
>     > > Hey Guozhang,
>     > >
>     > > Responses below:
>     > >
>     > > Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP.
>     > >
>     > >
>     > > I was considering an extension to support pre-initialization of the
>     > static
>     > > members of the group, but I agree we should probably leave this problem
>     > for
>     > > future work.
>     > >
>     > > 1. How this longish static member expiration timeout defined? Is it via a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >
>     > >
>     > > I am not too sure. I tend to lean toward server-side configs because they
>     > > are easier to evolve. If we have to add something to the protocol, then
>     > > we'll be stuck with it forever.
>     > >
>     > > 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >
>     > >
>     > > Yeah, I hope we can ultimately get rid of it, but we may need it for
>     > > compatibility with older brokers. A related question is what should be
>     > the
>     > > behavior of the consumer if `member.name` is provided but the broker
>     > does
>     > > not support it? We could either fail or silently downgrade to dynamic
>     > > membership.
>     > >
>     > > -Jason
>     > >
>     > >
>     > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <wa...@gmail.com>
>     > wrote:
>     > >>
>     > >> Hey Jason,
>     > >>
>     > >> I like your idea to simplify the upgrade protocol to allow co-exist of
>     > >> static and dynamic members. Admittedly it may make the coordinator-side
>     > >> logic a bit more complex, but I think it worth doing it.
>     > >>
>     > >> Originally I was trying to kill more birds with one stone with KIP-345,
>     > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
>     > >> multi-instance client (mentioned as case 1)/2) in my early email), and
>     > >> hence proposing to have a pure static-membership protocol. But thinking
>     > >> twice about it I now feel it may be too ambitious and worth fixing in
>     > >> another KIP. With that, I think what you've proposed here is a good way
>     > to
>     > >> go for KIP-345 itself.
>     > >>
>     > >> Note there are a few details in your proposal we'd still need to figure
>     > >> out:
>     > >>
>     > >> 1. How this longish static member expiration timeout defined? Is it via
>     > a
>     > >> broker, hence global config, or via a client config which can be
>     > >> communicated to broker via JoinGroupRequest?
>     > >>
>     > >> 2. Assuming that for static members, LEAVE_GROUP request will not
>     > trigger a
>     > >> rebalance immediately either, similar to session timeout, but only the
>     > >> longer member expiration timeout, can we remove the internal "
>     > >> internal.leave.group.on.close" config, which is a quick walk-around
>     > then?
>     > >>
>     > >>
>     > >>
>     > >> Guozhang
>     > >>
>     > >>
>     > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <ja...@confluent.io>
>     > >> wrote:
>     > >>
>     > >>> Hey All,
>     > >>>
>     > >>> Nice to see some solid progress on this. It sounds like one of the
>     > >>> complications is allowing static and dynamic registration to coexist.
>     > I'm
>     > >>> wondering if we can do something like the following:
>     > >>>
>     > >>> 1. Statically registered members (those joining the group with a
>     > >> non-null `
>     > >>> member.name`) maintain a session with the coordinator just like
>     > dynamic
>     > >>> members.
>     > >>> 2. If a session is active for a static member when a rebalance begins,
>     > >> then
>     > >>> basically we'll keep the current behavior. The rebalance will await the
>     > >>> static member joining the group.
>     > >>> 3. If a static member does not have an active session, then the
>     > >> coordinator
>     > >>> will not wait for it to join, but will still include it in the
>     > rebalance.
>     > >>> The coordinator will forward the cached subscription information to the
>     > >>> leader and will cache the assignment after the rebalance completes.
>     > (Note
>     > >>> that we still have the generationId to fence offset commits from a
>     > static
>     > >>> zombie if the assignment changes.)
>     > >>> 4. When a static member leaves the group or has its session expire, no
>     > >>> rebalance is triggered. Instead, we can begin a timer to expire the
>     > >> static
>     > >>> registration. This would be a longish timeout (like 30 minutes say).
>     > >>>
>     > >>> So basically static members participate in all rebalances regardless
>     > >>> whether they have an active session. In a given rebalance, some of the
>     > >>> members may be static and some dynamic. The group leader can
>     > >> differentiate
>     > >>> the two based on the presence of the `member.name` (we have to add
>     > this
>     > >> to
>     > >>> the JoinGroupResponse). Generally speaking, we would choose leaders
>     > >>> preferentially from the active members that support the latest
>     > JoinGroup
>     > >>> protocol and are using static membership. If we have to choose a leader
>     > >>> with an old version, however, it would see all members in the group
>     > >> (static
>     > >>> or dynamic) as dynamic members and perform the assignment as usual.
>     > >>>
>     > >>> Would that work?
>     > >>>
>     > >>> -Jason
>     > >>>
>     > >>>
>     > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <wa...@gmail.com>
>     > >> wrote:
>     > >>>
>     > >>>> Hello Boyang,
>     > >>>>
>     > >>>> Thanks for the updated proposal, a few questions:
>     > >>>>
>     > >>>> 1. Where will "change-group-timeout" be communicated to the broker?
>     > >> Will
>     > >>>> that be a new field in the JoinGroupRequest, or are we going to
>     > >>> piggy-back
>     > >>>> on the existing session-timeout field (assuming that the original
>     > value
>     > >>>> will not be used anywhere in the static membership any more)?
>     > >>>>
>     > >>>> 2. "However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we shall still trigger rebalance but it could still try to catch
>     > >>>> `change-group-timeout`.": what does this mean? I thought your proposal
>     > >> is
>     > >>>> that for static memberships, the broker will NOT trigger rebalance
>     > even
>     > >>>> after session-timeout has been detected, but only that after
>     > >>>> change-group-timeout
>     > >>>> which is supposed to be longer than session-timeout to be defined?
>     > >>>>
>     > >>>> 3. "A join group request with member.name set will be treated as
>     > >>>> `static-membership` strategy", in this case, how would the switch from
>     > >>>> dynamic to static happen, since whoever changed the member.name to
>     > >>>> not-null
>     > >>>> will be rejected, right?
>     > >>>>
>     > >>>> 4. "just erase the cached mapping, and wait for session timeout to
>     > >>> trigger
>     > >>>> rebalance should be sufficient." this is also a bit unclear to me: who
>     > >>> will
>     > >>>> erase the cached mapping? Since it is on the broker-side I assume that
>     > >>>> broker has to do it. Are you suggesting to use a new request for it?
>     > >>>>
>     > >>>> 5. "Halfway switch": following 3) above, if your proposal is basically
>     > >> to
>     > >>>> let "first join-request wins", and the strategy will stay as is until
>     > >> all
>     > >>>> members are gone, then this will also not happen since whoever used
>     > >>>> different strategy as the first guy who sends join-group request will
>     > >> be
>     > >>>> rejected right?
>     > >>>>
>     > >>>>
>     > >>>> Guozhang
>     > >>>>
>     > >>>>
>     > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <jo...@confluent.io>
>     > >> wrote:
>     > >>>>
>     > >>>>> This sounds good to me!
>     > >>>>>
>     > >>>>> Thanks for the time you've spent on it,
>     > >>>>> -John
>     > >>>>>
>     > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <bc...@outlook.com>
>     > >>>> wrote:
>     > >>>>>
>     > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
>     > >> haven't
>     > >>>> got
>     > >>>>>> time to update this thread. To summarize what we come up so far,
>     > >> here
>     > >>>> is
>     > >>>>> a
>     > >>>>>> draft updated plan:
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Introduce a new config called `member.name` which is supposed to
>     > >> be
>     > >>>>>> provided uniquely by the consumer client. The broker will maintain
>     > >> a
>     > >>>>> cache
>     > >>>>>> with [key:member.name, value:member.id]. A join group request with
>     > >>>>>> member.name set will be treated as `static-membership` strategy,
>     > >> and
>     > >>>>> will
>     > >>>>>> reject any join group request without member.name. So this
>     > >>>> coordination
>     > >>>>>> change will be differentiated from the `dynamic-membership`
>     > >> protocol
>     > >>> we
>     > >>>>>> currently have.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling static join group request:
>     > >>>>>>
>     > >>>>>>  1.   The broker will check the membership to see whether this is
>     > >> a
>     > >>>> new
>     > >>>>>> member. If new, broker allocate a unique member id, cache the
>     > >> mapping
>     > >>>> and
>     > >>>>>> move to rebalance stage.
>     > >>>>>>  2.   Following 1, if this is an existing member, broker will not
>     > >>>> change
>     > >>>>>> group state, and return its cached member.id and current
>     > >> assignment.
>     > >>>>>> (unless this is leader, we shall trigger rebalance)
>     > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with pair
>     > >>> member
>     > >>>>>> name and id, I think for join group request it is ok to leave
>     > >> member
>     > >>> id
>     > >>>>>> blank as member name is the unique identifier. In commit offset
>     > >>> request
>     > >>>>> we
>     > >>>>>> *must* have both.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When handling commit offset request, if enabled with static
>     > >>> membership,
>     > >>>>>> each time the commit request must have both member.name and
>     > >>> member.id
>     > >>>> to
>     > >>>>>> be identified as a `certificated member`. If not, this means there
>     > >>> are
>     > >>>>>> duplicate consumer members with same member name and the request
>     > >> will
>     > >>>> be
>     > >>>>>> rejected to guarantee consumption uniqueness.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> When rolling restart/shutting down gracefully, the client will
>     > >> send a
>     > >>>>>> leave group request (static membership mode). In static membership,
>     > >>> we
>     > >>>>> will
>     > >>>>>> also define `change-group-timeout` to hold on rebalance provided by
>     > >>>>> leader.
>     > >>>>>> So we will wait for all the members to rejoin the group and do
>     > >>> exactly
>     > >>>>> one
>     > >>>>>> rebalance since all members are expected to rejoin within timeout.
>     > >> If
>     > >>>>>> consumer crashes, the join group request from the restarted
>     > >> consumer
>     > >>>> will
>     > >>>>>> be recognized as an existing member and be handled as above
>     > >> condition
>     > >>>> 1;
>     > >>>>>> However, if the consumer takes longer than session timeout to
>     > >> return,
>     > >>>> we
>     > >>>>>> shall still trigger rebalance but it could still try to catch
>     > >>>>>> `change-group-timeout`. If it failed to catch second timeout, its
>     > >>>> cached
>     > >>>>>> state on broker will be garbage collected and trigger a new
>     > >> rebalance
>     > >>>>> when
>     > >>>>>> it finally joins.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> And consider the switch between dynamic to static membership.
>     > >>>>>>
>     > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
>     > >> membership
>     > >>>> to
>     > >>>>>> static and wait for all the current members to restart, since their
>     > >>>>>> membership is still dynamic. Here our assumption is that the
>     > >> restart
>     > >>>>>> process shouldn't take a long time, as long restart is breaking the
>     > >>>>>> `rebalance timeout` in whatever membership protocol we are using.
>     > >>>> Before
>     > >>>>>> restart, all dynamic member join requests will be rejected.
>     > >>>>>>  2.  Static to dynamic: this is more like a downgrade which should
>     > >>> be
>     > >>>>>> smooth: just erase the cached mapping, and wait for session timeout
>     > >>> to
>     > >>>>>> trigger rebalance should be sufficient. (Fallback to current
>     > >>> behavior)
>     > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
>     > >> dynamic
>     > >>>>>> membership while some keep static membership. This will cause the
>     > >>> group
>     > >>>>>> rebalance forever without progress because dynamic/static states
>     > >> are
>     > >>>>>> bouncing each other. This could guarantee that we will not make the
>     > >>>>>> consumer group work in a wrong state by having half static and half
>     > >>>>> dynamic.
>     > >>>>>>
>     > >>>>>> To guarantee correctness, we will also push the member name/id pair
>     > >>> to
>     > >>>>>> _consumed_offsets topic (as Matthias pointed out) and upgrade the
>     > >> API
>     > >>>>>> version, these details will be further discussed back in the KIP.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Are there any concern for this high level proposal? Just want to
>     > >>>>> reiterate
>     > >>>>>> on the core idea of the KIP: "If the broker recognize this consumer
>     > >>> as
>     > >>>> an
>     > >>>>>> existing member, it shouldn't trigger rebalance".
>     > >>>>>>
>     > >>>>>> Thanks a lot for everyone's input! I feel this proposal is much
>     > >> more
>     > >>>>>> robust than previous one!
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> Best,
>     > >>>>>>
>     > >>>>>> Boyang
>     > >>>>>>
>     > >>>>>> ________________________________
>     > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
>     > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
>     > >>>>>> To: dev@kafka.apache.org
>     > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
>     > >>> by
>     > >>>>>> specifying member id
>     > >>>>>>
>     > >>>>>> Hi,
>     > >>>>>>
>     > >>>>>> thanks for the detailed discussion. I learned a lot about internals
>     > >>>> again
>     > >>>>>> :)
>     > >>>>>>
>     > >>>>>> I like the idea or a user config `member.name` and to keep `
>     > >>> member.id`
>     > >>>>>> internal. Also agree with Guozhang, that reusing `client.id` might
>     > >>> not
>     > >>>>>> be a good idea.
>     > >>>>>>
>     > >>>>>> To clarify the algorithm, each time we generate a new `member.id`,
>     > >>> we
>     > >>>>>> also need to update the "group membership" information (ie, mapping
>     > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id` replaces
>     > >>> the
>     > >>>>>> old entry in the cache.
>     > >>>>>>
>     > >>>>>> I also think, we need to preserve the `member.name -> member.id`
>     > >>>> mapping
>     > >>>>>> in the `__consumer_offset` topic. The KIP should mention this IMHO.
>     > >>>>>>
>     > >>>>>> For changing the default value of config `leave.group.on.close`. I
>     > >>>> agree
>     > >>>>>> with John, that we should not change the default config, because it
>     > >>>>>> would impact all consumer groups with dynamic assignment. However,
>     > >> I
>     > >>>>>> think we can document, that if static assignment is used (ie,
>     > >>>>>> `member.name` is configured) we never send a LeaveGroupRequest
>     > >>>>>> regardless of the config. Note, that the config is internal, so not
>     > >>>> sure
>     > >>>>>> how to document this in detail. We should not expose the internal
>     > >>>> config
>     > >>>>>> in the docs.
>     > >>>>>>
>     > >>>>>> About upgrading: why do we need have two rolling bounces and encode
>     > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
>     > >>>>>>
>     > >>>>>> If we upgrade an existing consumer group from dynamic to static, I
>     > >>>> don't
>     > >>>>>> see any reason why both should not work together and single rolling
>     > >>>>>> bounce would not be sufficient? If we bounce the first consumer and
>     > >>>>>> switch from dynamic to static, it sends a `member.name` and the
>     > >>> broker
>     > >>>>>> registers the [member.name, member.id] in the cache. Why would
>     > >> this
>     > >>>>>> interfere with all other consumer that use dynamic assignment?
>     > >>>>>>
>     > >>>>>> Also, Guozhang mentioned that for all other request, we need to
>     > >> check
>     > >>>> if
>     > >>>>>> the mapping [member.name, member.id] contains the send `member.id`
>     > >>> --
>     > >>>> I
>     > >>>>>> don't think this is necessary -- it seems to be sufficient to check
>     > >>> the
>     > >>>>>> `member.id` from the [member.id, Assignment] mapping as be do
>     > >> today
>     > >>> --
>     > >>>>>> thus, checking `member.id` does not require any change IMHO.
>     > >>>>>>
>     > >>>>>>
>     > >>>>>> -Matthias
>     > >>>>>>
>     > >>>>>>
>     > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
>     > >>>>>>> @James
>     > >>>>>>>
>     > >>>>>>> What you described is true: the transition from dynamic to static
>     > >>>>>>> memberships are not thought through yet. But I do not think it is
>     > >>> an
>     > >>>>>>> impossible problem: note that we indeed moved the offset commit
>     > >>> from
>     > >>>> ZK
>     > >>>>>> to
>     > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to first to
>     > >>>>>>> double-commits on both zk and coordinator, and then do a second
>     > >>> round
>     > >>>>> to
>     > >>>>>>> turn the zk off.
>     > >>>>>>>
>     > >>>>>>> So just to throw a wild idea here: also following a
>     > >>>> two-rolling-bounce
>     > >>>>>>> manner, in the JoinGroupRequest we can set the flag to "static"
>     > >>> while
>     > >>>>>> keep
>     > >>>>>>> the registry-id field empty still, in this case, the coordinator
>     > >>>> still
>     > >>>>>>> follows the logic of "dynamic", accepting the request while
>     > >>> allowing
>     > >>>>> the
>     > >>>>>>> protocol to be set to "static"; after the first rolling bounce,
>     > >> the
>     > >>>>> group
>     > >>>>>>> protocol is already "static", then a second rolling bounce is
>     > >>>> triggered
>     > >>>>>> and
>     > >>>>>>> this time we set the registry-id.
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>> Guozhang
>     > >>>>>>>
>     > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
>     > >> wushujames@gmail.com>
>     > >>>>>> wrote:
>     > >>>>>>>
>     > >>>>>>>> Guozhang, in a previous message, you proposed said this:
>     > >>>>>>>>
>     > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <wangguoz@gmail.com
>     > >>>
>     > >>>>> wrote:
>     > >>>>>>>>>
>     > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional fields:
>     > >>>>>>>>>
>     > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
>     > >>> protocols.
>     > >>>>>>>>> 1.b) with "static" membership, we also add the pre-defined
>     > >>> member
>     > >>>>> id.
>     > >>>>>>>>> 1.c) with "static" membership, we also add an optional
>     > >>>>>>>>> "group-change-timeout" value.
>     > >>>>>>>>>
>     > >>>>>>>>> 2. On the broker side, we enforce only one of the two protocols
>     > >>> for
>     > >>>>> all
>     > >>>>>>>>> group members: we accept the protocol on the first joined
>     > >> member
>     > >>> of
>     > >>>>> the
>     > >>>>>>>>> group, and if later joining members indicate a different
>     > >>> membership
>     > >>>>>>>>> protocol, we reject it. If the group-change-timeout value was
>     > >>>>> different
>     > >>>>>>>> to
>     > >>>>>>>>> the first joined member, we reject it as well.
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>> What will happen if we have an already-deployed application that
>     > >>>> wants
>     > >>>>>> to
>     > >>>>>>>> switch to using static membership? Let’s say there are 10
>     > >>> instances
>     > >>>> of
>     > >>>>>> it.
>     > >>>>>>>> As the instances go through a rolling restart, they will switch
>     > >>> from
>     > >>>>>>>> dynamic membership (the default?) to static membership. As each
>     > >>> one
>     > >>>>>> leaves
>     > >>>>>>>> the group and restarts, they will be rejected from the group
>     > >>>> (because
>     > >>>>>> the
>     > >>>>>>>> group is currently using dynamic membership). The group will
>     > >>> shrink
>     > >>>>> down
>     > >>>>>>>> until there is 1 node handling all the traffic. After that one
>     > >>>>> restarts,
>     > >>>>>>>> the group will switch over to static membership.
>     > >>>>>>>>
>     > >>>>>>>> Is that right? That means that the transition plan from dynamic
>     > >> to
>     > >>>>>> static
>     > >>>>>>>> membership isn’t very smooth.
>     > >>>>>>>>
>     > >>>>>>>> I’m not really sure what can be done in this case. This reminds
>     > >> me
>     > >>>> of
>     > >>>>>> the
>     > >>>>>>>> transition plans that were discussed for moving from
>     > >>> zookeeper-based
>     > >>>>>>>> consumers to kafka-coordinator-based consumers. That was also
>     > >>> hard,
>     > >>>>> and
>     > >>>>>>>> ultimately we decided not to build that.
>     > >>>>>>>>
>     > >>>>>>>> -James
>     > >>>>>>>>
>     > >>>>>>>>
>     > >>>>>>>
>     > >>>>>>>
>     > >>>>>>
>     > >>>>>>
>     > >>>>>
>     > >>>>
>     > >>>>
>     > >>>>
>     > >>>> --
>     > >>>> -- Guozhang
>     > >>>>
>     > >>>
>     > >>
>     > >>
>     > >>
>     > >> --
>     > >> -- Guozhang
>     > >>
>     >
> 
> 


Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Posted by Boyang Chen <bc...@outlook.com>.
Hey Mike,


thanks for the feedback, the two question are very thoughtful!


> 1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I > do not think a bounce of the leader should trigger a rebalance.

For Q1 my intention was to minimize the change within one KIP, since the leader rejoining case could be addressed separately.


> 2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes > to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to > do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on > shrink down. What do you think?

For 2) my understanding is that for scaling down case it is better to be addressed by CLI tool than code logic, since only by human evaluation we could decide whether it is a "right timing" -- the time when all the scaling down consumers are offline -- to kick in rebalance. Unless we introduce another term on coordinator which indicates the target consumer group size, broker will find it hard to decide when to start rebalance. So far I prefer to hold the implementation for that, but agree we could discuss whether we want to introduce admin API in this KIP or a separate one.


Thanks again for the proposed ideas!


Boyang

________________________________
From: Mike Freyberger <mi...@xandr.com>
Sent: Monday, November 5, 2018 6:13 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

Boyang,

Thanks for updating the KIP. It's shaping up well. Two things:

1) I am a little confused about the distinction for the leader. If the consumer node that was assigned leader does a bounce (goes down and quickly comes up) to update application code, will a rebalance be triggered? I do not think a bounce of the leader should trigger a rebalance.

2) The timeout for shrink up makes a lot of sense and allows to gracefully increase the number of nodes in the cluster. I think we need to support graceful shrink down as well. If I set the registration timeout to 5 minutes to handle rolling restarts or intermittent failures without shuffling state, I don't want to wait 5 minutes in order for the group to rebalance if I am intentionally removing a node from the cluster. I am not sure the best way to do this. One idea I had was adding the ability for a CLI or Admin API to force a rebalance of the group. This would allow for an admin to trigger the rebalance manually without waiting the entire registration timeout on shrink down. What do you think?

Mike

On 10/30/18, 1:55 AM, "Boyang Chen" <bc...@outlook.com> wrote:

    Btw, I updated KIP 345 based on my understanding. Feel free to take another round of look:

    https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances
KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
cwiki.apache.org
For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...




    KIP-345: Introduce static membership protocol to reduce ...<https://cwiki.apache.org/confluence/display/KAFKA/KIP-345%3A+Introduce+static+membership+protocol+to+reduce+consumer+rebalances>
    cwiki.apache.org
    For stateful applications, one of the biggest performance bottleneck is the state shuffling. In Kafka consumer, there is a concept called "rebalance" which means that for given M partitions and N consumers in one consumer group, Kafka will try to balance the load between consumers and ideally have ...





    ________________________________
    From: Boyang Chen <bc...@outlook.com>
    Sent: Monday, October 29, 2018 12:34 PM
    To: dev@kafka.apache.org
    Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

    Thanks everyone for the input on this thread! (Sorry it's been a while) I feel that we are very close to the final solution.


    Hey Jason and Mike, I have two quick questions on the new features here:

      1.  so our proposal is that until we add a new static member into the group (scale up), we will not trigger rebalance until the "registration timeout"( the member has been offline for too long)? How about leader's rejoin request, I think we should still trigger rebalance when that happens, since the consumer group may have new topics to consume?
      2.  I'm not very clear on the scale up scenario in static membership here. Should we fallback to dynamic membership while adding/removing hosts (by setting member.name = null), or we still want to add instances with `member.name` so that we eventually expand/shrink the static membership? I personally feel the easier solution is to spin up new members and wait until either the same "registration timeout" or a "scale up timeout" before starting the rebalance. What do you think?

    Meanwhile I will go ahead to make changes to the KIP with our newly discussed items and details. Really excited to see the design has become more solid.

    Best,
    Boyang

    ________________________________
    From: Jason Gustafson <ja...@confluent.io>
    Sent: Saturday, August 25, 2018 6:04 AM
    To: dev
    Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by specifying member id

    Hey Mike,

    Yeah, that's a good point. A long "registration timeout" may not be a great
    idea. Perhaps in practice you'd set it long enough to be able to detect a
    failure and provision a new instance. Maybe on the order of 10 minutes is
    more reasonable.

    In any case, it's probably a good idea to have an administrative way to
    force deregistration. One option is to extend the DeleteGroups API with a
    list of members names.

    -Jason



    On Fri, Aug 24, 2018 at 2:21 PM, Mike Freyberger <mf...@appnexus.com>
    wrote:

    > Jason,
    >
    > Regarding step 4 in your proposal which suggests beginning a long timer
    > (30 minutes) when a static member leaves the group, would there also be the
    > ability for an admin to force a static membership expiration?
    >
    > I’m thinking that during particular types of outages or upgrades users
    > would want forcefully remove a static member from the group.
    >
    > So the user would shut the consumer down normally, which wouldn’t trigger
    > a rebalance. Then the user could use an admin CLI tool to force remove that
    > consumer from the group, so the TopicPartitions that were previously owned
    > by that consumer can be released.
    >
    > At a high level, we need consumer groups to gracefully handle intermittent
    > failures and permanent failures. Currently, the consumer group protocol
    > handles permanent failures well, but does not handle intermittent failures
    > well (it creates unnecessary rebalances). I want to make sure the overall
    > solution here handles both intermittent failures and permanent failures,
    > rather than sacrificing support for permanent failures in order to provide
    > support for intermittent failures.
    >
    > Mike
    >
    > Sent from my iPhone
    >
    > > On Aug 24, 2018, at 3:03 PM, Jason Gustafson <ja...@confluent.io> wrote:
    > >
    > > Hey Guozhang,
    > >
    > > Responses below:
    > >
    > > Originally I was trying to kill more birds with one stone with KIP-345,
    > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
    > >> multi-instance client (mentioned as case 1)/2) in my early email), and
    > >> hence proposing to have a pure static-membership protocol. But thinking
    > >> twice about it I now feel it may be too ambitious and worth fixing in
    > >> another KIP.
    > >
    > >
    > > I was considering an extension to support pre-initialization of the
    > static
    > > members of the group, but I agree we should probably leave this problem
    > for
    > > future work.
    > >
    > > 1. How this longish static member expiration timeout defined? Is it via a
    > >> broker, hence global config, or via a client config which can be
    > >> communicated to broker via JoinGroupRequest?
    > >
    > >
    > > I am not too sure. I tend to lean toward server-side configs because they
    > > are easier to evolve. If we have to add something to the protocol, then
    > > we'll be stuck with it forever.
    > >
    > > 2. Assuming that for static members, LEAVE_GROUP request will not
    > trigger a
    > >> rebalance immediately either, similar to session timeout, but only the
    > >> longer member expiration timeout, can we remove the internal "
    > >> internal.leave.group.on.close" config, which is a quick walk-around
    > then?
    > >
    > >
    > > Yeah, I hope we can ultimately get rid of it, but we may need it for
    > > compatibility with older brokers. A related question is what should be
    > the
    > > behavior of the consumer if `member.name` is provided but the broker
    > does
    > > not support it? We could either fail or silently downgrade to dynamic
    > > membership.
    > >
    > > -Jason
    > >
    > >
    > >> On Fri, Aug 24, 2018 at 11:44 AM, Guozhang Wang <wa...@gmail.com>
    > wrote:
    > >>
    > >> Hey Jason,
    > >>
    > >> I like your idea to simplify the upgrade protocol to allow co-exist of
    > >> static and dynamic members. Admittedly it may make the coordinator-side
    > >> logic a bit more complex, but I think it worth doing it.
    > >>
    > >> Originally I was trying to kill more birds with one stone with KIP-345,
    > >> e.g. to fix the multi-rebalance issue on starting up / shutting down a
    > >> multi-instance client (mentioned as case 1)/2) in my early email), and
    > >> hence proposing to have a pure static-membership protocol. But thinking
    > >> twice about it I now feel it may be too ambitious and worth fixing in
    > >> another KIP. With that, I think what you've proposed here is a good way
    > to
    > >> go for KIP-345 itself.
    > >>
    > >> Note there are a few details in your proposal we'd still need to figure
    > >> out:
    > >>
    > >> 1. How this longish static member expiration timeout defined? Is it via
    > a
    > >> broker, hence global config, or via a client config which can be
    > >> communicated to broker via JoinGroupRequest?
    > >>
    > >> 2. Assuming that for static members, LEAVE_GROUP request will not
    > trigger a
    > >> rebalance immediately either, similar to session timeout, but only the
    > >> longer member expiration timeout, can we remove the internal "
    > >> internal.leave.group.on.close" config, which is a quick walk-around
    > then?
    > >>
    > >>
    > >>
    > >> Guozhang
    > >>
    > >>
    > >> On Fri, Aug 24, 2018 at 11:14 AM, Jason Gustafson <ja...@confluent.io>
    > >> wrote:
    > >>
    > >>> Hey All,
    > >>>
    > >>> Nice to see some solid progress on this. It sounds like one of the
    > >>> complications is allowing static and dynamic registration to coexist.
    > I'm
    > >>> wondering if we can do something like the following:
    > >>>
    > >>> 1. Statically registered members (those joining the group with a
    > >> non-null `
    > >>> member.name`) maintain a session with the coordinator just like
    > dynamic
    > >>> members.
    > >>> 2. If a session is active for a static member when a rebalance begins,
    > >> then
    > >>> basically we'll keep the current behavior. The rebalance will await the
    > >>> static member joining the group.
    > >>> 3. If a static member does not have an active session, then the
    > >> coordinator
    > >>> will not wait for it to join, but will still include it in the
    > rebalance.
    > >>> The coordinator will forward the cached subscription information to the
    > >>> leader and will cache the assignment after the rebalance completes.
    > (Note
    > >>> that we still have the generationId to fence offset commits from a
    > static
    > >>> zombie if the assignment changes.)
    > >>> 4. When a static member leaves the group or has its session expire, no
    > >>> rebalance is triggered. Instead, we can begin a timer to expire the
    > >> static
    > >>> registration. This would be a longish timeout (like 30 minutes say).
    > >>>
    > >>> So basically static members participate in all rebalances regardless
    > >>> whether they have an active session. In a given rebalance, some of the
    > >>> members may be static and some dynamic. The group leader can
    > >> differentiate
    > >>> the two based on the presence of the `member.name` (we have to add
    > this
    > >> to
    > >>> the JoinGroupResponse). Generally speaking, we would choose leaders
    > >>> preferentially from the active members that support the latest
    > JoinGroup
    > >>> protocol and are using static membership. If we have to choose a leader
    > >>> with an old version, however, it would see all members in the group
    > >> (static
    > >>> or dynamic) as dynamic members and perform the assignment as usual.
    > >>>
    > >>> Would that work?
    > >>>
    > >>> -Jason
    > >>>
    > >>>
    > >>> On Thu, Aug 23, 2018 at 5:26 PM, Guozhang Wang <wa...@gmail.com>
    > >> wrote:
    > >>>
    > >>>> Hello Boyang,
    > >>>>
    > >>>> Thanks for the updated proposal, a few questions:
    > >>>>
    > >>>> 1. Where will "change-group-timeout" be communicated to the broker?
    > >> Will
    > >>>> that be a new field in the JoinGroupRequest, or are we going to
    > >>> piggy-back
    > >>>> on the existing session-timeout field (assuming that the original
    > value
    > >>>> will not be used anywhere in the static membership any more)?
    > >>>>
    > >>>> 2. "However, if the consumer takes longer than session timeout to
    > >> return,
    > >>>> we shall still trigger rebalance but it could still try to catch
    > >>>> `change-group-timeout`.": what does this mean? I thought your proposal
    > >> is
    > >>>> that for static memberships, the broker will NOT trigger rebalance
    > even
    > >>>> after session-timeout has been detected, but only that after
    > >>>> change-group-timeout
    > >>>> which is supposed to be longer than session-timeout to be defined?
    > >>>>
    > >>>> 3. "A join group request with member.name set will be treated as
    > >>>> `static-membership` strategy", in this case, how would the switch from
    > >>>> dynamic to static happen, since whoever changed the member.name to
    > >>>> not-null
    > >>>> will be rejected, right?
    > >>>>
    > >>>> 4. "just erase the cached mapping, and wait for session timeout to
    > >>> trigger
    > >>>> rebalance should be sufficient." this is also a bit unclear to me: who
    > >>> will
    > >>>> erase the cached mapping? Since it is on the broker-side I assume that
    > >>>> broker has to do it. Are you suggesting to use a new request for it?
    > >>>>
    > >>>> 5. "Halfway switch": following 3) above, if your proposal is basically
    > >> to
    > >>>> let "first join-request wins", and the strategy will stay as is until
    > >> all
    > >>>> members are gone, then this will also not happen since whoever used
    > >>>> different strategy as the first guy who sends join-group request will
    > >> be
    > >>>> rejected right?
    > >>>>
    > >>>>
    > >>>> Guozhang
    > >>>>
    > >>>>
    > >>>> On Tue, Aug 21, 2018 at 9:28 AM, John Roesler <jo...@confluent.io>
    > >> wrote:
    > >>>>
    > >>>>> This sounds good to me!
    > >>>>>
    > >>>>> Thanks for the time you've spent on it,
    > >>>>> -John
    > >>>>>
    > >>>>> On Tue, Aug 21, 2018 at 12:13 AM Boyang Chen <bc...@outlook.com>
    > >>>> wrote:
    > >>>>>
    > >>>>>> Thanks Matthias for the input. Sorry I was busy recently and
    > >> haven't
    > >>>> got
    > >>>>>> time to update this thread. To summarize what we come up so far,
    > >> here
    > >>>> is
    > >>>>> a
    > >>>>>> draft updated plan:
    > >>>>>>
    > >>>>>>
    > >>>>>> Introduce a new config called `member.name` which is supposed to
    > >> be
    > >>>>>> provided uniquely by the consumer client. The broker will maintain
    > >> a
    > >>>>> cache
    > >>>>>> with [key:member.name, value:member.id]. A join group request with
    > >>>>>> member.name set will be treated as `static-membership` strategy,
    > >> and
    > >>>>> will
    > >>>>>> reject any join group request without member.name. So this
    > >>>> coordination
    > >>>>>> change will be differentiated from the `dynamic-membership`
    > >> protocol
    > >>> we
    > >>>>>> currently have.
    > >>>>>>
    > >>>>>>
    > >>>>>> When handling static join group request:
    > >>>>>>
    > >>>>>>  1.   The broker will check the membership to see whether this is
    > >> a
    > >>>> new
    > >>>>>> member. If new, broker allocate a unique member id, cache the
    > >> mapping
    > >>>> and
    > >>>>>> move to rebalance stage.
    > >>>>>>  2.   Following 1, if this is an existing member, broker will not
    > >>>> change
    > >>>>>> group state, and return its cached member.id and current
    > >> assignment.
    > >>>>>> (unless this is leader, we shall trigger rebalance)
    > >>>>>>  3.   Although Guozhang has mentioned we could rejoin with pair
    > >>> member
    > >>>>>> name and id, I think for join group request it is ok to leave
    > >> member
    > >>> id
    > >>>>>> blank as member name is the unique identifier. In commit offset
    > >>> request
    > >>>>> we
    > >>>>>> *must* have both.
    > >>>>>>
    > >>>>>>
    > >>>>>> When handling commit offset request, if enabled with static
    > >>> membership,
    > >>>>>> each time the commit request must have both member.name and
    > >>> member.id
    > >>>> to
    > >>>>>> be identified as a `certificated member`. If not, this means there
    > >>> are
    > >>>>>> duplicate consumer members with same member name and the request
    > >> will
    > >>>> be
    > >>>>>> rejected to guarantee consumption uniqueness.
    > >>>>>>
    > >>>>>>
    > >>>>>> When rolling restart/shutting down gracefully, the client will
    > >> send a
    > >>>>>> leave group request (static membership mode). In static membership,
    > >>> we
    > >>>>> will
    > >>>>>> also define `change-group-timeout` to hold on rebalance provided by
    > >>>>> leader.
    > >>>>>> So we will wait for all the members to rejoin the group and do
    > >>> exactly
    > >>>>> one
    > >>>>>> rebalance since all members are expected to rejoin within timeout.
    > >> If
    > >>>>>> consumer crashes, the join group request from the restarted
    > >> consumer
    > >>>> will
    > >>>>>> be recognized as an existing member and be handled as above
    > >> condition
    > >>>> 1;
    > >>>>>> However, if the consumer takes longer than session timeout to
    > >> return,
    > >>>> we
    > >>>>>> shall still trigger rebalance but it could still try to catch
    > >>>>>> `change-group-timeout`. If it failed to catch second timeout, its
    > >>>> cached
    > >>>>>> state on broker will be garbage collected and trigger a new
    > >> rebalance
    > >>>>> when
    > >>>>>> it finally joins.
    > >>>>>>
    > >>>>>>
    > >>>>>> And consider the switch between dynamic to static membership.
    > >>>>>>
    > >>>>>>  1.  Dynamic to static: the first joiner shall revise the
    > >> membership
    > >>>> to
    > >>>>>> static and wait for all the current members to restart, since their
    > >>>>>> membership is still dynamic. Here our assumption is that the
    > >> restart
    > >>>>>> process shouldn't take a long time, as long restart is breaking the
    > >>>>>> `rebalance timeout` in whatever membership protocol we are using.
    > >>>> Before
    > >>>>>> restart, all dynamic member join requests will be rejected.
    > >>>>>>  2.  Static to dynamic: this is more like a downgrade which should
    > >>> be
    > >>>>>> smooth: just erase the cached mapping, and wait for session timeout
    > >>> to
    > >>>>>> trigger rebalance should be sufficient. (Fallback to current
    > >>> behavior)
    > >>>>>>  3.  Halfway switch: a corner case is like some clients keep
    > >> dynamic
    > >>>>>> membership while some keep static membership. This will cause the
    > >>> group
    > >>>>>> rebalance forever without progress because dynamic/static states
    > >> are
    > >>>>>> bouncing each other. This could guarantee that we will not make the
    > >>>>>> consumer group work in a wrong state by having half static and half
    > >>>>> dynamic.
    > >>>>>>
    > >>>>>> To guarantee correctness, we will also push the member name/id pair
    > >>> to
    > >>>>>> _consumed_offsets topic (as Matthias pointed out) and upgrade the
    > >> API
    > >>>>>> version, these details will be further discussed back in the KIP.
    > >>>>>>
    > >>>>>>
    > >>>>>> Are there any concern for this high level proposal? Just want to
    > >>>>> reiterate
    > >>>>>> on the core idea of the KIP: "If the broker recognize this consumer
    > >>> as
    > >>>> an
    > >>>>>> existing member, it shouldn't trigger rebalance".
    > >>>>>>
    > >>>>>> Thanks a lot for everyone's input! I feel this proposal is much
    > >> more
    > >>>>>> robust than previous one!
    > >>>>>>
    > >>>>>>
    > >>>>>> Best,
    > >>>>>>
    > >>>>>> Boyang
    > >>>>>>
    > >>>>>> ________________________________
    > >>>>>> From: Matthias J. Sax <ma...@confluent.io>
    > >>>>>> Sent: Friday, August 10, 2018 2:24 AM
    > >>>>>> To: dev@kafka.apache.org
    > >>>>>> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances
    > >>> by
    > >>>>>> specifying member id
    > >>>>>>
    > >>>>>> Hi,
    > >>>>>>
    > >>>>>> thanks for the detailed discussion. I learned a lot about internals
    > >>>> again
    > >>>>>> :)
    > >>>>>>
    > >>>>>> I like the idea or a user config `member.name` and to keep `
    > >>> member.id`
    > >>>>>> internal. Also agree with Guozhang, that reusing `client.id` might
    > >>> not
    > >>>>>> be a good idea.
    > >>>>>>
    > >>>>>> To clarify the algorithm, each time we generate a new `member.id`,
    > >>> we
    > >>>>>> also need to update the "group membership" information (ie, mapping
    > >>>>>> [member.id, Assignment]), right? Ie, the new `member.id` replaces
    > >>> the
    > >>>>>> old entry in the cache.
    > >>>>>>
    > >>>>>> I also think, we need to preserve the `member.name -> member.id`
    > >>>> mapping
    > >>>>>> in the `__consumer_offset` topic. The KIP should mention this IMHO.
    > >>>>>>
    > >>>>>> For changing the default value of config `leave.group.on.close`. I
    > >>>> agree
    > >>>>>> with John, that we should not change the default config, because it
    > >>>>>> would impact all consumer groups with dynamic assignment. However,
    > >> I
    > >>>>>> think we can document, that if static assignment is used (ie,
    > >>>>>> `member.name` is configured) we never send a LeaveGroupRequest
    > >>>>>> regardless of the config. Note, that the config is internal, so not
    > >>>> sure
    > >>>>>> how to document this in detail. We should not expose the internal
    > >>>> config
    > >>>>>> in the docs.
    > >>>>>>
    > >>>>>> About upgrading: why do we need have two rolling bounces and encode
    > >>>>>> "static" vs "dynamic" in the JoinGroupRequest?
    > >>>>>>
    > >>>>>> If we upgrade an existing consumer group from dynamic to static, I
    > >>>> don't
    > >>>>>> see any reason why both should not work together and single rolling
    > >>>>>> bounce would not be sufficient? If we bounce the first consumer and
    > >>>>>> switch from dynamic to static, it sends a `member.name` and the
    > >>> broker
    > >>>>>> registers the [member.name, member.id] in the cache. Why would
    > >> this
    > >>>>>> interfere with all other consumer that use dynamic assignment?
    > >>>>>>
    > >>>>>> Also, Guozhang mentioned that for all other request, we need to
    > >> check
    > >>>> if
    > >>>>>> the mapping [member.name, member.id] contains the send `member.id`
    > >>> --
    > >>>> I
    > >>>>>> don't think this is necessary -- it seems to be sufficient to check
    > >>> the
    > >>>>>> `member.id` from the [member.id, Assignment] mapping as be do
    > >> today
    > >>> --
    > >>>>>> thus, checking `member.id` does not require any change IMHO.
    > >>>>>>
    > >>>>>>
    > >>>>>> -Matthias
    > >>>>>>
    > >>>>>>
    > >>>>>>> On 8/7/18 7:13 PM, Guozhang Wang wrote:
    > >>>>>>> @James
    > >>>>>>>
    > >>>>>>> What you described is true: the transition from dynamic to static
    > >>>>>>> memberships are not thought through yet. But I do not think it is
    > >>> an
    > >>>>>>> impossible problem: note that we indeed moved the offset commit
    > >>> from
    > >>>> ZK
    > >>>>>> to
    > >>>>>>> kafka coordinator in 0.8.2 :) The migration plan is to first to
    > >>>>>>> double-commits on both zk and coordinator, and then do a second
    > >>> round
    > >>>>> to
    > >>>>>>> turn the zk off.
    > >>>>>>>
    > >>>>>>> So just to throw a wild idea here: also following a
    > >>>> two-rolling-bounce
    > >>>>>>> manner, in the JoinGroupRequest we can set the flag to "static"
    > >>> while
    > >>>>>> keep
    > >>>>>>> the registry-id field empty still, in this case, the coordinator
    > >>>> still
    > >>>>>>> follows the logic of "dynamic", accepting the request while
    > >>> allowing
    > >>>>> the
    > >>>>>>> protocol to be set to "static"; after the first rolling bounce,
    > >> the
    > >>>>> group
    > >>>>>>> protocol is already "static", then a second rolling bounce is
    > >>>> triggered
    > >>>>>> and
    > >>>>>>> this time we set the registry-id.
    > >>>>>>>
    > >>>>>>>
    > >>>>>>> Guozhang
    > >>>>>>>
    > >>>>>>> On Tue, Aug 7, 2018 at 1:19 AM, James Cheng <
    > >> wushujames@gmail.com>
    > >>>>>> wrote:
    > >>>>>>>
    > >>>>>>>> Guozhang, in a previous message, you proposed said this:
    > >>>>>>>>
    > >>>>>>>>> On Jul 30, 2018, at 3:56 PM, Guozhang Wang <wangguoz@gmail.com
    > >>>
    > >>>>> wrote:
    > >>>>>>>>>
    > >>>>>>>>> 1. We bump up the JoinGroupRequest with additional fields:
    > >>>>>>>>>
    > >>>>>>>>> 1.a) a flag indicating "static" or "dynamic" membership
    > >>> protocols.
    > >>>>>>>>> 1.b) with "static" membership, we also add the pre-defined
    > >>> member
    > >>>>> id.
    > >>>>>>>>> 1.c) with "static" membership, we also add an optional
    > >>>>>>>>> "group-change-timeout" value.
    > >>>>>>>>>
    > >>>>>>>>> 2. On the broker side, we enforce only one of the two protocols
    > >>> for
    > >>>>> all
    > >>>>>>>>> group members: we accept the protocol on the first joined
    > >> member
    > >>> of
    > >>>>> the
    > >>>>>>>>> group, and if later joining members indicate a different
    > >>> membership
    > >>>>>>>>> protocol, we reject it. If the group-change-timeout value was
    > >>>>> different
    > >>>>>>>> to
    > >>>>>>>>> the first joined member, we reject it as well.
    > >>>>>>>>
    > >>>>>>>>
    > >>>>>>>> What will happen if we have an already-deployed application that
    > >>>> wants
    > >>>>>> to
    > >>>>>>>> switch to using static membership? Let’s say there are 10
    > >>> instances
    > >>>> of
    > >>>>>> it.
    > >>>>>>>> As the instances go through a rolling restart, they will switch
    > >>> from
    > >>>>>>>> dynamic membership (the default?) to static membership. As each
    > >>> one
    > >>>>>> leaves
    > >>>>>>>> the group and restarts, they will be rejected from the group
    > >>>> (because
    > >>>>>> the
    > >>>>>>>> group is currently using dynamic membership). The group will
    > >>> shrink
    > >>>>> down
    > >>>>>>>> until there is 1 node handling all the traffic. After that one
    > >>>>> restarts,
    > >>>>>>>> the group will switch over to static membership.
    > >>>>>>>>
    > >>>>>>>> Is that right? That means that the transition plan from dynamic
    > >> to
    > >>>>>> static
    > >>>>>>>> membership isn’t very smooth.
    > >>>>>>>>
    > >>>>>>>> I’m not really sure what can be done in this case. This reminds
    > >> me
    > >>>> of
    > >>>>>> the
    > >>>>>>>> transition plans that were discussed for moving from
    > >>> zookeeper-based
    > >>>>>>>> consumers to kafka-coordinator-based consumers. That was also
    > >>> hard,
    > >>>>> and
    > >>>>>>>> ultimately we decided not to build that.
    > >>>>>>>>
    > >>>>>>>> -James
    > >>>>>>>>
    > >>>>>>>>
    > >>>>>>>
    > >>>>>>>
    > >>>>>>
    > >>>>>>
    > >>>>>
    > >>>>
    > >>>>
    > >>>>
    > >>>> --
    > >>>> -- Guozhang
    > >>>>
    > >>>
    > >>
    > >>
    > >>
    > >> --
    > >> -- Guozhang
    > >>
    >