Hi, Andrew, Thanks for the reply. Just a minor comment.
JR26. "The keys of the map are either single versions like "0" or closed version ranges like "1-2"." It will be useful to support open-ended versions. That way, "3": "3" could be "3+": "3". This part won't need to change in the future when we bump up the request version without changing the header. Jun On Fri, Jun 12, 2026 at 2:12 PM Andrew Schofield <[email protected]> wrote: > Hi Jun, > Thanks for your response. > > JR25: I agree. I've changed to a headerVersions map in both the request > and response schemas. > > Let me know what you think. > > Thanks, > Andrew > > On 2026/06/12 03:39:39 Jun Rao via dev wrote: > > Hi, Andrew, > > > > Thanks for the updated KIP. > > > > JR25. I have some concerns about adding the clientInstanceIdVersions > field > > to the schema. First of all, it only applies to requests, but not > > responses. So, it's a bit weird to add it to the general schema. > Secondly, > > it's very specific to the new field added to the request header. If we > > change the request header again in the future, we need another way to map > > request versions to the new header version. Ideally, we should have a > > generic way to map request versions to all future header versions. One > > possibility is to introduce a headerVersion field in the schema. Then, > each > > request/response can use this field to map its versions to the header > > versions. > > > > Jun > > > > On Tue, Jun 9, 2026 at 4:55 AM Andrew Schofield <[email protected]> > > wrote: > > > > > Hi Matthias, > > > Thanks for your response. > > > > > > I have change the version property for introducing client instance ID > into > > > the request headers to be `clientInstanceIdVersions` which is now an > > > open-ended range, such as "3+", more closely matching what we have for > > > flexible versions. > > > > > > I also added a change to the RebalanceConsumer.clientInstanceId method > > > introduced in KIP-1306. > > > > > > Thanks, > > > Andrew > > > > > > On 2026/06/03 18:31:06 "Matthias J. Sax" wrote: > > > > Thanks Andrew. > > > > > > > > From a KIP-714 perspective, this is a great change, as it make the > new > > > > RPCs and client/broker cross-version compatibility clean. > > > > > > > > One question about your `DeleteGroups` RPC example. Should the new > > > > version of the RPC say > > > > > > > > > "clientInstanceIdVersion": "3+", > > > > > > > > ie, 3+ instead of 3? Similar for other request, which don't use `+` > on > > > > `clientInstanceIdVersion` field. > > > > > > > > > > > > > > > > -Matthias > > > > > > > > > > > > > > > > On 6/2/26 7:42 AM, Andrew Schofield wrote: > > > > > Hi, > > > > > Following on from the discussion in the community, and the > difficulty > > > of trying to accommodate the client instance ID in a tagged field with > > > acceptable complexity, I have made a significant change to the KIP. > Now, it > > > introduces v3 of the request header containing the client instance ID > as a > > > proper field. > > > > > > > > > > > > > > https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/KAFKA/KIP-1313*3A*Client*instance*ID*in*all*request*headers__;JSsrKysrKys!!Ayb5sqE7!pUZS25PuTKdhiqvTGcwDKbclqpG8dFOzw2Ax9WxVM6Sb09PVQ0gl1i0gjlr2_Krr66agbuVC2qLGgUcg7vrW$ > > > > > > > > > > Please review and let me know what you think. > > > > > > > > > > Thanks, > > > > > Andrew > > > > > > > > > > On 2026/05/28 21:51:01 "Matthias J. Sax" wrote: > > > > >> Thanks Andrew. I agree to the issue that tagged fields are > optional. > > > > >> > > > > >> Nevertheless, I find the proposed compatibility protocol too > complex > > > and > > > > >> unnecessary, and would still prefer a version bump for > > > > >> `GetTelemetrySubscriptionsRequest/Reponse` to clean it all up. -- > For > > > > >> `PushTelemetryRequest` the argument from above applies, so > leaving it > > > > >> untouched is ok. > > > > >> > > > > >> I don't see why `GetTelemetrySubscriptionsRequest` would need to > > > encode > > > > >> the `clientInstandId` in it's request body. Given that the > > > > >> `clientInstanceId` is a random UUID, the returned subscription > cannot > > > > >> really depend on it. So why would we need to send it to the > broker? > > > > >> Right now, it's set to ZERO only for the purpose to get the > broker to > > > > >> assign a new UUID. But it has nothing to do with the actual "what > > > > >> metrics should I send" question. > > > > >> > > > > >> To be fair, KIP-714 lists `client_instance_id` as a field that > can be > > > > >> used to define a subscription. But it seems rather useless in > practice > > > > >> to me? I a user defines a subscription, they cannot know the UUID. > > > Thus, > > > > >> I think we should actually drop supporting `client_instance_id` > as a > > > > >> "subscription matching parameter"? Of course, there is a backward > > > > >> compatibility question, but I think we can address this: if > brokers > > > are > > > > >> upgraded and they have an existing `client_instance_id` based > > > > >> subscription defined, they could advertise to only support v0, and > > > they > > > > >> should log a warning that this feature is deprecated. New brokers > > > would > > > > >> also not allow to use `client_instance_id` any longer to define a > > > > >> subscription. > > > > >> > > > > >> Similarly, for `GetTelemetrySubscriptionsResponse`, we only need > > > > >> `clientInstanceId` if the broker computes the UUID. But if we let > the > > > > >> client compute it, this field is not needed any longer. > > > > >> > > > > >> > > > > >> -Matthias > > > > >> > > > > >> > > > > >> > > > > >> On 5/28/26 1:21 AM, Andrew Schofield wrote: > > > > >>> Hi Jun, > > > > >>> Thanks for your response. > > > > >>> > > > > >>> JR24: I'll update the KIP shortly. Here's a summary. The various > > > modern group protocol heartbeat requests encode the member ID as a > string > > > for historical reasons, even though it's in practice a UUID. The Apache > > > Kafka Java client happens to use org.apache.kafka.common.Uuid to > create the > > > member ID and convert it into a string. This means that when the > > > clientInstanceId is converted into a string, its encoding matches the > > > member ID and it all lines up. However, there are other ways to encode > a > > > UUID into a string and I know for a fact that the librdkafka client > does it > > > differently. > > > > >>> > > > > >>> Thanks, > > > > >>> Andrew > > > > >>> > > > > >>> On 2026/05/27 17:11:25 Jun Rao via dev wrote: > > > > >>>> Hi, Andrew, > > > > >>>> > > > > >>>> Thanks for the reply. One more comment. > > > > >>>> > > > > >>>> JR24. "The alignment of other identifiers is by convention (and > the > > > Java > > > > >>>> client will follow the convention) rather than mandate." Could > you > > > describe > > > > >>>> the convention to convert a clientInstanceId (UUID) to a > memberId > > > (String)? > > > > >>>> > > > > >>>> Jun > > > > >>>> > > > > >>>> > > > > >>>> On Tue, May 19, 2026 at 2:36 AM Andrew Schofield < > > > [email protected]> > > > > >>>> wrote: > > > > >>>> > > > > >>>>> Hi Jun, > > > > >>>>> Thanks for your response. > > > > >>>>> > > > > >>>>> JR23: You are absolutely correct. It seems to me that not > sending a > > > > >>>>> clientInstanceId in the header and explicitly sending a zero > UUID > > > as the > > > > >>>>> clientInstanceId in the header can be treated as semantically > > > equivalent. > > > > >>>>> I've tweaked the words slightly. > > > > >>>>> > > > > >>>>> Thanks, > > > > >>>>> Andrew > > > > >>>>> > > > > >>>>> On 2026/05/19 03:42:16 Jun Rao via dev wrote: > > > > >>>>>> Hi, Andrew, > > > > >>>>>> > > > > >>>>>> Thanks for the reply. > > > > >>>>>> > > > > >>>>>> JR23. Our message protocol doc says "Any fields in the message > > > object > > > > >>>>> that > > > > >>>>>> are not present in the version that you are deserializing > will be > > > reset > > > > >>>>> to > > > > >>>>>> default values. Unless a custom default has been set:". Uuid > > > fields > > > > >>>>>> default to zero uuid. > > > > >>>>>> So if the server gets header.clientInstanceId=0 in the > > > deserialized > > > > >>>>> header, > > > > >>>>>> could it distinguish between the ID not being present (since > > > client is > > > > >>>>> old) > > > > >>>>>> and the ID being explicitly set to 0 by the client? > > > > >>>>>> > > > > >>>>>> Jun > > > > >>>>>> > > > > >>>>>> > > > > >>>>>> On Mon, May 18, 2026 at 7:45 PM Andrew Schofield < > > > [email protected]> > > > > >>>>>> wrote: > > > > >>>>>> > > > > >>>>>>> Hi Jun, > > > > >>>>>>> Thanks for your reply. It's tricky squaring a circle. > > > > >>>>>>> > > > > >>>>>>> JR23: For GetTelemetrySubscriptions, I have changed it so > that a > > > client > > > > >>>>>>> which omits the ClientInstanceId from the request header is > > > permitted > > > > >>>>> to > > > > >>>>>>> specify a zero ClientInstanceId in the request body, > following > > > original > > > > >>>>>>> KIP-714 precedent. However, a client which specifies a > > > > >>>>> ClientInstanceId in > > > > >>>>>>> the request header MUST specify the same ClientInstanceId in > the > > > > >>>>> request > > > > >>>>>>> body. This ensures that the header and telemetry UUIDs are > the > > > same. > > > > >>>>>>> > > > > >>>>>>> Thanks, > > > > >>>>>>> Andrew > > > > >>>>>>> > > > > >>>>>>> On 2026/05/12 17:48:23 Andrew Schofield wrote: > > > > >>>>>>>> Hi Jun, > > > > >>>>>>>> Thanks for the reply and digging into the details. > > > > >>>>>>>> > > > > >>>>>>>> JR23: Correct. The client telemetry component will use > UUID-B > > > as the > > > > >>>>>>> client instance ID. > > > > >>>>>>>> > > > > >>>>>>>> JR23.1: Yes, I agree. It's not ideal. When I was drawing up > the > > > > >>>>> tables, > > > > >>>>>>> I was thinking that this might be a possibility, but I'm less > > > convinced > > > > >>>>>>> now. I think that I should mandate that if a client specifies > > > > >>>>>>> header.ClientInstanceId on GetTelemetrySubscriptions request, > > > then > > > > >>>>>>> request.ClientInstanceId must either be zero or equal to > > > > >>>>>>> header.ClientInstanceId. > > > > >>>>>>>> > > > > >>>>>>>> JR23.2: This is perhaps the interesting one. From its > original > > > > >>>>> intent, > > > > >>>>>>> it should be UUID-B (the telemetry UUID), but then that > > > contradicts the > > > > >>>>>>> change in signature to remove the timeout. Unless I make the > > > change > > > > >>>>> above, > > > > >>>>>>> in which case it will be UUID-H. > > > > >>>>>>>> > > > > >>>>>>>> Thanks, > > > > >>>>>>>> Andrew > > > > >>>>>>>> > > > > >>>>>>>> On 2026/05/12 17:23:58 Jun Rao via dev wrote: > > > > >>>>>>>>> Hi, Andrew, > > > > >>>>>>>>> > > > > >>>>>>>>> Thanks for the reply. > > > > >>>>>>>>> > > > > >>>>>>>>> JR23. In the new client -> old broker case, we have > > > > >>>>>>>>> header.ClientInstanceId=UUID-H > > > > >>>>>>>>> request.ClientInstanceId=UUID-B > > > > >>>>>>>>> response.ClientInstanceId=0 > > > > >>>>>>>>> > > > > >>>>>>>>> On the server side, I guess the telemetry component will > use > > > > >>>>> UUID-B as > > > > >>>>>>> the > > > > >>>>>>>>> clientInstanceId? This has a couple of implications. > > > > >>>>>>>>> JR23.1 On the server side, we have two different > > > clientInstanceIds > > > > >>>>>>> used in > > > > >>>>>>>>> different places, UUID-H for request logging and UUID-B in > > > > >>>>> telemetry. > > > > >>>>>>> This > > > > >>>>>>>>> seems confusing since we can't uniquely identify a client > on > > > the > > > > >>>>> server > > > > >>>>>>>>> side. > > > > >>>>>>>>> JR23.2 On the client side. what uuid does > > > clientInstanceId(Duration > > > > >>>>>>>>> timeout) return? If it returns UUID-H, it will be confusing > > > since > > > > >>>>> it > > > > >>>>>>>>> doesn't match the ID used for telemetry on the server. > > > > >>>>>>>>> > > > > >>>>>>>>> Jun > > > > >>>>>>>>> > > > > >>>>>>>>> > > > > >>>>>>>>> > > > > >>>>>>>>> On Tue, May 12, 2026 at 12:58 AM Andrew Schofield < > > > > >>>>>>> [email protected]> > > > > >>>>>>>>> wrote: > > > > >>>>>>>>> > > > > >>>>>>>>>> Hi Jun, > > > > >>>>>>>>>> Thanks for your response. > > > > >>>>>>>>>> > > > > >>>>>>>>>> JR20: I have improved (I hope) the wording. The client > sends > > > > >>>>>>>>>> request.clientInstanceId = 0 and header.clientInstanceId = > > > > >>>>> UUID-H, > > > > >>>>>>> and the > > > > >>>>>>>>>> broker responds response.clientInstanceId=UUID-H. In this > way, > > > > >>>>> the > > > > >>>>>>> broker > > > > >>>>>>>>>> will have taken the UUID-H from the header, and told the > > > client > > > > >>>>> to > > > > >>>>>>> use it > > > > >>>>>>>>>> for client telemetry also. > > > > >>>>>>>>>> > > > > >>>>>>>>>> JR21: Done. Look for "henceforth". > > > > >>>>>>>>>> > > > > >>>>>>>>>> JR22: Summary table added. > > > > >>>>>>>>>> > > > > >>>>>>>>>> Thanks, > > > > >>>>>>>>>> Andrew > > > > >>>>>>>>>> > > > > >>>>>>>>>> On 2026/05/11 19:18:24 Jun Rao via dev wrote: > > > > >>>>>>>>>>> Hi, Andrew, > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> Thanks for the reply. > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> JR20. "If the client requests a new client instance ID > on its > > > > >>>>>>> initial > > > > >>>>>>>>>>> GetTelemetrySubscriptions request and it sends a client > > > > >>>>> instance > > > > >>>>>>> ID in > > > > >>>>>>>>>> the > > > > >>>>>>>>>>> request header, the broker will send back that client > > > instance > > > > >>>>> ID > > > > >>>>>>> rather > > > > >>>>>>>>>>> than generating a new UUID. This will automatically > align the > > > > >>>>> UUID > > > > >>>>>>> in the > > > > >>>>>>>>>>> request headers and client telemetry." > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> This seems inconsistent with what's in the table. In the > > > > >>>>> table, for > > > > >>>>>>>>>>> example, if the client has the following: > > > > >>>>>>>>>>> GetTelemetrySubscriptions v0 > > > > >>>>>>>>>>> header.ClientInstanceId = UUID-H > > > > >>>>>>>>>>> request.ClientInstanceId = UUID-H > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> or > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> GetTelemetrySubscriptions v0 > > > > >>>>>>>>>>> header.ClientInstanceId = UUID-H > > > > >>>>>>>>>>> request.ClientInstanceId = UUID-R > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> the broker returns > > > > >>>>>>>>>>> response.ClientInstanceId = 0. > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> JR21. It will be useful to document what the new client > does > > > > >>>>> with > > > > >>>>>>> the > > > > >>>>>>>>>>> returned response.ClientInstanceId. Note that return > value > > > may > > > > >>>>> or > > > > >>>>>>> may not > > > > >>>>>>>>>>> be 0. > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> JR22. It's probably clearer if we could populate the > table > > > > >>>>> with 4 > > > > >>>>>>>>>>> combinations: old/new clients with old/new brokers. > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> Jun > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> > > > > >>>>>>>>>>> On Fri, May 8, 2026 at 2:49 AM Andrew Schofield < > > > > >>>>>>> [email protected]> > > > > >>>>>>>>>>> wrote: > > > > >>>>>>>>>>> > > > > >>>>>>>>>>>> Hi Jun and Chia-Ping, > > > > >>>>>>>>>>>> I've overhauled part of the KIP to do with alignment of > the > > > > >>>>>>> request > > > > >>>>>>>>>> header > > > > >>>>>>>>>>>> client instance ID, client telemetry client instance ID > and > > > > >>>>> group > > > > >>>>>>>>>> protocol > > > > >>>>>>>>>>>> member IDs. The alignment is by convention, not mandate > > > > >>>>> (SHOULD > > > > >>>>>>> not > > > > >>>>>>>>>> MUST). > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>>> It would be possible to go around the existing RPCs > such as > > > > >>>>>>>>>>>> ConsumerGroupHeartbeat and GetTelemetrySubscriptions, > and > > > > >>>>> remove > > > > >>>>>>> the > > > > >>>>>>>>>> fields > > > > >>>>>>>>>>>> containing the existing identifiers which are intended > to be > > > > >>>>>>> aligned. > > > > >>>>>>>>>> Doing > > > > >>>>>>>>>>>> so would be a bad idea though, because we would then > have > > > RPC > > > > >>>>>>> versions > > > > >>>>>>>>>>>> which essentially depend upon the presence of a tagged > field > > > > >>>>> in > > > > >>>>>>> the > > > > >>>>>>>>>> request > > > > >>>>>>>>>>>> header. This is a protocol-compatibility nightmare. > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>>> I have removed the new versions of > GetTelemetrySubscriptions > > > > >>>>> and > > > > >>>>>>>>>>>> PushTelemetry. I have also explained the behavior of > > > > >>>>>>>>>>>> GetTelemetrySubscriptions in the presence and absence > of a > > > > >>>>> client > > > > >>>>>>>>>> instance > > > > >>>>>>>>>>>> ID in the request header. > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>>> Let me know what you think. > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>>> Thanks, > > > > >>>>>>>>>>>> Andrew > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>>> On 2026/05/07 15:09:31 Andrew Schofield wrote: > > > > >>>>>>>>>>>>> Hi Jun and Chia-Ping, > > > > >>>>>>>>>>>>> I've been thinking and discussing the changes to the > > > > >>>>> KIP-714 > > > > >>>>>>> RPCs. > > > > >>>>>>>>>> There > > > > >>>>>>>>>>>> are too many combinations for my liking at the moment. I > > > > >>>>> want to > > > > >>>>>>> take > > > > >>>>>>>>>>>> another pass at this area and will make an update in a > few > > > > >>>>> days. > > > > >>>>>>>>>>>>> > > > > >>>>>>>>>>>>> I intend to start a new vote once we have consensus > because > > > > >>>>>>> the spec > > > > >>>>>>>>>> has > > > > >>>>>>>>>>>> changed somewhat since the earliest votes. > > > > >>>>>>>>>>>>> > > > > >>>>>>>>>>>>> Thanks, > > > > >>>>>>>>>>>>> Andrew > > > > >>>>>>>>>>>>> > > > > >>>>>>>>>>>>> On 2026/05/06 17:28:27 Chia-Ping Tsai wrote: > > > > >>>>>>>>>>>>>> hi Andrew > > > > >>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>> chia_0: If the consensus is to remove the "duplicate" > > > > >>>>> field > > > > >>>>>>> from > > > > >>>>>>>>>> the > > > > >>>>>>>>>>>> RPC payloads, the tagged field in the header will > > > essentially > > > > >>>>>>> become a > > > > >>>>>>>>>>>> required field. This means the broker needs to handle > the > > > > >>>>> edge > > > > >>>>>>> case > > > > >>>>>>>>>> where > > > > >>>>>>>>>>>> both the header and the request body have no > > > > >>>>> ClientInstanceId, > > > > >>>>>>> right? > > > > >>>>>>>>>> If > > > > >>>>>>>>>>>> so, would you mind clarifying the expected broker > behavior > > > in > > > > >>>>>>> the KIP? > > > > >>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>> Best, > > > > >>>>>>>>>>>>>> Chia-Ping > > > > >>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>> On 2026/04/03 16:17:37 Andrew Schofield wrote: > > > > >>>>>>>>>>>>>>> Hi, > > > > >>>>>>>>>>>>>>> I would like to start the discussion on KIP-1313. > This > > > > >>>>>>> adds a > > > > >>>>>>>>>> unique > > > > >>>>>>>>>>>> client instance ID to the request header of all Kafka > > > > >>>>> protocol > > > > >>>>>>>>>> requests to > > > > >>>>>>>>>>>> give a unique identifier which can be used to correlate > the > > > > >>>>>>> requests > > > > >>>>>>>>>> from > > > > >>>>>>>>>>>> each client for the purposes of problem determination. > > > > >>>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>>> > > > > >>>>>>>>>>>> > > > > >>>>>>>>>> > > > > >>>>>>> > > > > >>>>> > > > > https://urldefense.com/v3/__https://cwiki.apache.org/confluence/display/KAFKA/KIP-1313*3A*Client*instance*ID*in*all*request*headers__;JSsrKysrKys!!Ayb5sqE7!uqWf0-b_X82WmpmCYImD2W2rht_s_q5vHcqB9ToMV4IaeQbZF42eMJyS5XC5b5qE_qJJUj3KTCXcqEvYbwYS$ > > > > >>>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>>> Thanks, > > > > >>>>>>>>>>>>>>> Andrew > > > > >>>>>>>>>>>>>>> > > > > >>>>>>>>>>>>>> > > > > >>>>>>>>>>>>> > > > > >>>>>>>>>>>> > > > > >>>>>>>>>>> > > > > >>>>>>>>>> > > > > >>>>>>>>> > > > > >>>>>>>> > > > > >>>>>>> > > > > >>>>>> > > > > >>>>> > > > > >>>> > > > > >> > > > > >> > > > > > > > > > > > > > >
