Hi Mickael, thanks for the comments.

MM1: Yes I’d left the implementation detail in the public API section from the 
old version. I’ve moved it.

MM2: Good call, I’ve added the behaviour as Javadoc in the interface section.

Thanks,
Aditya

On 2026/05/26 13:53:06 Mickael Maison wrote:
> Hi,
>
> Thanks for the updates. That seems reasonable improvement to the header APIs.
>
> MM1: In the Public Interfaces section, the RecordHeader class is still
> mentioned. Is it a mistake? I don't think we want to expose that
> class.
>
> MM2: It seems you defined the behavior of the adders and accessors in
> the Proposed Changes section. Could you add that as javadoc in the
> Public Interfaces section?
>
> Thanks,
> Mickael
>
>
>
>
>
>
> On Tue, May 26, 2026 at 6:33 AM Aditya Kousik <[email protected]> wrote:
> >
> > Hey folks,
> >
> > Quick follow-up on this KIP. The changes proposed are all client side QoL 
> > updates for easier access to the Header interface(s) without any change to 
> > bytes on the wire.
> >
> > Best,
> > Aditya
> >
> > > On May 11, 2026, at 18:50, Aditya Kousik <[email protected]> wrote:
> > >
> > > Hi all,
> > > Gentle ping on KIP-1308.
> > >
> > > Even in the simple case of logging a header value you already know the 
> > > type of, you’re still reaching for ByteBuffer/new String() today:
> > >
> > > // Today
> > > int retryCount = ByteBuffer.wrap(header.value()).getInt();
> > > log.debug(“retryCount={}”, retryCount);
> > >
> > > // With KIP-1308
> > > log.debug(“retryCount={}”, header.intValue());
> > >
> > > Looking forward to your thoughts.
> > >
> > > Thanks,
> > > Aditya
> > >
> > >> On Apr 16, 2026, at 12:04, Aditya Kousik <[email protected]> wrote:
> > >>
> > >> Hi Mickael and Andrew,
> > >>
> > >> I’ve updated the KIP to reflect new methods on the Header(s) interface 
> > >> and new static factory methods.
> > >>
> > >> Please take a look when you can.
> > >>
> > >> Thanks,
> > >> Aditya
> > >>
> > >>>> On Apr 3, 2026, at 08:13, Aditya Kousik <[email protected]> wrote:
> > >>>
> > >>> Hi Mickael,
> > >>>
> > >>> That’s a solid point. I realise there are others like me who have been 
> > >>> accessing the RecordHeader class directly 
> > >>> (https://github.com/spring-projects/spring-kafka/blob/813b9af333a6b812617e970f34ca0ae447dda7fb/spring-kafka/src/main/java/org/springframework/kafka/listener/KafkaMessageListenerContainer.java#L2812)
> > >>>  by virtue of it being public and well documented.
> > >>>
> > >>> I don’t want to break the precedence and make RecordHeader part of the 
> > >>> public API at this point, but I like your idea of adding addXXX-like 
> > >>> methods to Headers.java. This way the KIP adds new methods to both 
> > >>> RecordHeader(s) and Header(s) with the helper methods. I’ve updated the 
> > >>> KIP to reflect this.
> > >>>
> > >>> Best,
> > >>> Aditya
> > >>>
> > >>>>> On 2026/04/02 14:56:36 Mickael Maison wrote:
> > >>>> Hi,
> > >>>>
> > >>>> Thanks for the KIP. I'm a bit confused because RecordHeader is not
> > >>>> part of the public API. It's in
> > >>>> org.apache.kafka.common.header.internals.
> > >>>> To create a Header, users have to create their own Header
> > >>>> implementation, or more likely use Headers.add(String key, byte[]
> > >>>> value).
> > >>>>
> > >>>> So either this would require making RecordHeader part of the public
> > >>>> API, or an alternative is to add Headers.add() overloads that take
> > >>>> common types as the 2nd argument.
> > >>>>
> > >>>> Thanks,
> > >>>> Mickael
> > >>>>
> > >>>>> On Wed, Apr 1, 2026 at 7:05 PM Andrew Schofield <[email protected]> 
> > >>>>> wrote:
> > >>>>>
> > >>>>> Hi
> > >>>>> Thanks for the KIPs. You're writing them faster than the community 
> > >>>>> can read them :)
> > >>>>>
> > >>>>> I'll take a proper look soon.
> > >>>>>
> > >>>>> Thanks,
> > >>>>> Andrew
> > >>>>>
> > >>>>> On 2026/04/01 15:35:47 Aditya Kousik wrote:
> > >>>>>> Hi all,
> > >>>>>>
> > >>>>>> I'd like to start a discussion on KIP-1308: Extend RecordHeader API 
> > >>>>>> for Common Value Types.
> > >>>>>>
> > >>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1308%3A+Extend+RecordHeader+API+for+Common+Value+Types
> > >>>>>>
> > >>>>>> Putting a string or int into a RecordHeader today requires manual 
> > >>>>>> byte encoding; reading it back requires the same in reverse. This 
> > >>>>>> KIP adds static factories (RecordHeader.ofString, ofInt, etc.) and 
> > >>>>>> corresponding typed accessors, with no wire format changes.
> > >>>>>>
> > >>>>>> Looking forward to your feedback.
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Aditya Kousik
> > >>>>>>
> > >>>>
> > >>
>

Reply via email to