> I think that this should be defined. Otherwise, folks may start using
incorrect things. In the java client, we will generate a Kafka UUID.
The javadoc of the Uuid class has a good definition for it.

That makes sense to me. However, there is no binding agreement when we only
put the recommendations. I means we can't stop folks from using incorrect
things unless the field "member id" is defined as UUID (for example, change
the protocol type from string to UUID)

> The proposal in
this KIP is to make the member id required in future versions. We could
also decide not to do it and to keep supporting both approaches. I would
also be fine with this.

IMHO, that is a kind of "incorrect thing". If we all agree the behavior of
this version could cause bugs (maybe the bugs are rare but they are indeed
existent), then it is worth having the new behavior to protect folks from
the possible bugs.

Best,
Chia-Ping

David Jacot <dja...@confluent.io.invalid> 於 2024年8月14日 週三 下午7:50寫道:

> Hi Andrew,
>
> Personally, I don't like the lobby approach. It makes things more
> complicated and it would require changing the records on the server too.
> This is why I initially suggested the rejected alternative #2 which is
> pretty close but also not perfect.
>
> I'd like to clarify one thing. The ConsumerGroupHeartbeat API already
> supports generating the member id on the client so we don't need any
> conditional logic on the client side. This is actually what we wanted to do
> in the first place but the idea got pushed back by Magnus back then because
> generating uuid from librdkafka required a new dependency. It turns out
> that librdkafka has that dependency today. In retrospect, we should have
> pushed back on this. Long story short, we can just do it. The proposal in
> this KIP is to make the member id required in future versions. We could
> also decide not to do it and to keep supporting both approaches. I would
> also be fine with this.
>
> Best,
> David
>
> On Wed, Aug 14, 2024 at 12:30 PM Andrew Schofield <
> andrew_schofi...@live.com>
> wrote:
>
> > Hi TengYao,
> > Thanks for your response. I’ll have just one more try to persuade.
> > I feel that I will need to follow the approach with KIP-932 when we’ve
> > made a decision, so I do have more than a passing interest in this.
> >
> > A group member in the lobby is in the group, but it does not have any
> > assignments. A member of a consumer group can have no assigned
> > partitions (such as 5 CG members subscribed to a topic with 4
> partitions),
> > so it’s a situation that consumer group members already expect.
> >
> > One of Kafka’s strengths is the way that we handle API versioning.
> > But, there is a cost - the behaviour is different depending on the RPC
> > version. KIP-848 is on the cusp of completion, but we’re already adding
> > conditional logic for v0/v1 for ConsumerGroupHeartbeat. That’s a pity.
> > Only a minor issue, but it’s unfortunate.
> >
> > Thanks,
> > Andrew
> >
> > > On 14 Aug 2024, at 08:47, TengYao Chi <kiting...@gmail.com> wrote:
> > >
> > > Hello Andrew
> > > Thank you for your thoughtful suggestions and getting the discussion
> > going.
> > >
> > > To AS1:
> > > In the current scenario where the server generates the UUID, if the
> > client
> > > shuts down before receiving the memberId generated by the GC
> (regardless
> > of
> > > whether it’s a graceful shutdown or not), the GC will still have to
> wait
> > > for the heartbeat timeout because the client doesn’t know its memberId.
> > > This KIP indeed cannot completely resolve the idempotency issue, but it
> > can
> > > better handle shutdown scenarios under normal circumstances because the
> > > client always knows its memberId. Even if the client shuts down
> > immediately
> > > after the initial heartbeat, as long as it performs a graceful shutdown
> > and
> > > sends a leave heartbeat, the GC can manage the situation and remove the
> > > member. Therefore, the goal of this KIP is to address the issue where
> the
> > > GC has to wait for the heartbeat timeout due to the client leaving
> > without
> > > knowing its memberId, which leads to reduced throughput and limited
> > > scalability.
> > >
> > > The solution you suggest has also been proposed by David. The concern
> > with
> > > this approach is that it introduces additional complexity for
> > > compatibility, as the new server would not immediately add the member
> to
> > > the group, while the old server would. This requires clients to
> > > differentiate whether their memberId has been added to the group or
> not,
> > > which could result in unexpected logs.
> > >
> > > Best Regards,
> > > TengYao
> > >
> > > Andrew Schofield <andrew_schofi...@live.com> 於 2024年8月14日 週三
> 上午12:29寫道:
> > >
> > >> Hi TengYao,
> > >> Thanks for the KIP. I wonder if there’s a different way to close what
> > >> is quite a small window.
> > >>
> > >> AS1: It is true that the initial heartbeat is not idempotent, but this
> > >> remains
> > >> true with this KIP. It’s just differently not idempotent. If the
> client
> > >> makes its
> > >> own member ID, sends a request and dies, the GC will still have added
> > >> the member to the group and it will hang around until the session
> > expires.
> > >>
> > >> I wonder if the GC could still generate the member ID in response to
> the
> > >> first
> > >> heartbeat, and put the member in a special PENDING state with no
> > >> assignments until the client sends the next heartbeat, thus confirming
> > it
> > >> has received the member ID. This would not be a protocol change at
> all,
> > >> just
> > >> a change to the GC to keep a new member in the lobby until it’s
> > comfirmed
> > >> it knows its member ID.
> > >>
> > >>
> > >> Thanks,
> > >> Andrew
> > >>
> > >>> On 13 Aug 2024, at 15:59, TengYao Chi <kiting...@gmail.com> wrote:
> > >>>
> > >>> Hi Chia-Ping,
> > >>>
> > >>> Thanks for review and suggestions.
> > >>> I have updated the content of KIP accordingly.
> > >>> Please take a look.
> > >>>
> > >>> Best regards,
> > >>> TengYao
> > >>>
> > >>> Chia-Ping Tsai <chia7...@apache.org> 於 2024年8月13日 週二 下午9:45寫道:
> > >>>
> > >>>> hi TengYao
> > >>>>
> > >>>> thanks for this KIP.
> > >>>>
> > >>>> 1) could you please describe the before/after behavior in the
> > "Proposed
> > >>>> Changes" section? IIRC, current RPC allows HB having member id
> > >> generated by
> > >>>> client, right? If HB has no member ID, server will generate one and
> > then
> > >>>> return. The new behavior will enforce HB "must" have member ID.
> > >>>>
> > >>>> 2) could you please write the version number explicitly in the KIP
> > >>>>
> > >>>> 3) how new client code handle the old HB? Does it always generate
> > member
> > >>>> ID on client-side even though that is not restricted?
> > >>>>
> > >>>> Best,
> > >>>> Chia-Ping
> > >>>>
> > >>>> On 2024/08/13 06:20:42 TengYao Chi wrote:
> > >>>>> Hello everyone,
> > >>>>>
> > >>>>> I would like to start a discussion thread on KIP-1082, which
> proposes
> > >>>>> enabling id generation for clients over the ConsumerGroupHeartbeat
> > RPC.
> > >>>>>
> > >>>>> Here is the KIP Link: KIP-1082
> > >>>>> <
> > >>>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1082%3A+Enable+ID+Generation+for+Clients+over+the+ConsumerGroupHeartbeat+RPC
> > >>>>>
> > >>>>> Please take a look and let me know what you think, and I would
> > >> appreciate
> > >>>>> any suggestions and feedback.
> > >>>>>
> > >>>>> Best regards,
> > >>>>> TengYao
> > >>>>>
> > >>>>
> > >>
> > >>
> >
> >
>

Reply via email to