Yep, that is correct!

Get Outlook for iOS<https://aka.ms/o0ukef>

________________________________
From: Mayuresh Gharat <gharatmayures...@gmail.com>
Sent: Wednesday, January 2, 2019 8:30 AM
To: dev@kafka.apache.org
Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by 
specifying member id

Hi Boyang,

Thanks a lot for the reply. I think, I get what you are saying.
IIUC, once the group starts moving to dynamic membership from static
membership, the group will have a list of new member.id's and the earlier
mapping of group.instance.id -> member.id will be outdated and become
inactive from pov of triggering rebalances. This is assuming that the time
interval to move from static to dynamic membership is small.

Thanks,

Mayuresh

On Fri, Dec 28, 2018 at 11:37 PM Boyang Chen <bche...@outlook.com> wrote:

> Hey Mayuresh,
>
> thanks for giving another thorough thought on this matter! The session
> timeout is controlled on the member level (or more specifically member.id
> level), which has nothing to do with the static member mapping. Putting
> aside the low possibility of having a duplicate member id assignment
> throughout the restart, if this indeed happens when we switch a member c1
> to dynamic member, the static membership map will only contain [gc1 -> mc1]
> mapping.
> This is because c2~c4 will be evicted during rebalance as their
> corresponding mc2~mc4 are already timeout through heartbeatpugatory. Next
> time the dynamic member restarts while group is in stable, the static map
> will be cleaned up and rebalance will be triggered as expected (we are
> already in dynamic mode now).
>
> I hope this answers your question, thanks!
>
> Boyang
>
> ________________________________
> From: Mayuresh Gharat <gharatmayures...@gmail.com>
> Sent: Saturday, December 22, 2018 2:21 AM
> To: dev@kafka.apache.org
> Subject: Re: [DISCUSS] KIP-345: Reduce multiple consumer rebalances by
> specifying member id
>
> Hi Boyang,
>
> Regarding "However, we shall still attempt to remove the member static info
> if the given `member.id` points to an existing `group.instance.id` upon
> LeaveGroupRequest, because I could think of the possibility that in long
> term we could want to add static membership leave group logic for more
> fine-grained use cases."
>
> > I think, there is some confusion here. I am probably not putting it
> > right.
> >
> I agree, If a static member sends LeaveGroupRequest, it should be removed
> > from the group.
> >
> Now getting back to downgrade of static membership to Dynamic membership,
> > with the example described earlier (copying it again for ease of
> reading)
> > :
> >
>
> >> 1. Lets say we have 4 consumers : c1, c2, c3, c4 in the static
> group.
> >> 2. The group.instance.id for each of there are as follows :
> >> - c1 -> gc1, c2 -> gc2, c3 -> gc3, c4 -> gc4
> >> 3. The mapping on the GroupCordinator would be :
> >> - gc1 -> mc1, gc2 -> mc2, gc3 -> mc3, gc4 -> mc4, where mc1, mc2,
> >> mc3, mc4 are the randomly generated memberIds for c1, c2, c3, c4
> >> respectively, by the GroupCoordinator.
> >> 4. Now we do a restart to move the group to dynamic membership.
> >> 5. We bounce c1 first and it rejoins with UNKNOWN_MEMBERID (since we
> >> don't persist the previously assigned memberId mc1 anywhere on the
> c1).
> >>
> > - We agree that there is no way to recognize that c1 was a part of the
> > group, *earlier*. If yes, the statement : "The dynamic member rejoins
> > the group without `group.instance.id`. It will be accepted since it is a
> > known member." is not necessarily true, right?
> >
>
>
> > - Now I *agree* with "However, we shall still attempt to remove the
> > member static info if the given `member.id` points to an existing `
> > group.instance.id` upon LeaveGroupRequest, because I could think of the
> > possibility that in long term we could want to add static membership
> leave
> > group logic for more fine-grained use cases."
> >
> But that would only happen if the GroupCoordinator allocates the same
> > member.id (mc1) to the consumer c1, when it rejoins the group in step 5
> > above as a dynamic member, which is very rare as it is randomly
> generated,
> > but possible.
> >
>
>
> > - This raises another question, if the GroupCoordinator assigns a
> > member.id (mc1~) to consumer c1 after step 5. It will join the group and
> > rebalance and the group will become stable, eventually. Now the
> > GroupCoordinator still maintains a mapping of "group.instance.id ->
> > member.id" (c1 -> gc1, c2 -> gc2, c3 -> gc3, c4 -> gc4) internally and
> > after some time, it realizes that it has not received heartbeat from the
> > consumer with "group.instance.id" = gc1. In that case, it will trigger
> > another rebalance assuming that a static member has left the group (when
> > actually it (c1) has not left the group but moved to dynamic membership).
> > This can result in multiple rebalances as the same will happen for c2,
> c3,
> > c4.
> >
>
> Thoughts ???
> One thing, I can think of right now is to run :
> removeMemberFromGroup(String groupId, list<String>
> groupInstanceIdsToRemove, RemoveMemberFromGroupOptions options)
> with groupInstanceIdsToRemove = <gc1, gc2, gc3, gc4> once we have bounced
> all the members in the group. This assumes that we will be able to complete
> the bounces before the GroupCoordinator realizes that it has not received a
> heartbeat for any of <gc1, gc2, gc3, gc4>. This is tricky and error prone.
> Will have to think more on this.
>
> Thanks,
>
> Mayuresh
>


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

Reply via email to