Hi TengYao,

I think this is a very useful feature for advanced applications that want to do 
buffer pooling, but cannot due to the current Serializer API accepting only 
byte arrays. Which I think is a pity as beyond the external interfaces 
(Serializer and Partitioner) most of the code can work with ByteBuffers.

I jumped the gun a little by already giving it a go based on the KIP you put 
together (thank you): https://github.com/apache/kafka/pull/21656 — so the 
proposed design definitely works. Based on that, I can also provide some 
feedback:

  - Serializer method naming: I would name the new method differently than the 
old one, since overloading by return type alone is not possible in Java, and 
changing parameter order may be confusing for users. I  think an explicit name 
such as serializeToByteBuffer would be more appropriate. Unless the intent is 
to eventually deprecate the byte[] variant and reclaim the serialize name for 
the new one?

  - Partitioner default implementation: The new ByteBuffer overload works well 
since the parameters are different, and providing a default implementation that 
delegates to the byte[] variant is straightforward. One note I find important 
here: when delegating, we should avoid allocating a new byte[] from the 
ByteBuffer when possible (i.e., return the backing array directly if it exactly 
matches the buffer contents). Users upgrading should not encounter an extra 
allocation that could cause performance regressions.

  - Internal ByteBuffer propagation: Something we could capture in the KIP is 
that to make the producer fully ByteBuffer-oriented, the built-in partitioner's 
murmur2 function needs to be updated to accept ByteBuffer directly. We could 
have used the same delegation strategy as with the Partitioner interface, but 
since this is not a public API with backward compatibility constraints, I think 
we can simply make it ByteBuffer-native.

Otherwise, I don't see anything outstanding. As mentioned in the KIP, this will 
enable better memory management for applications across the entire 
consume-to-produce flow, closing the cycle that KIP-863 started on the 
deserializer side.

Please let me know what you think.

Thanks,
Flore

On 2025/05/20 07:15:34 TengYao Chi wrote:
> Hi everyone,
> 
> I want to bump this thread manually.
> Thanks for your attention.
> 
> Best,
> TengYao
> 
> TengYao Chi <[email protected]> 於 2025年5月6日 週二 下午12:27寫道:
> 
> > Hello everyone,
> >
> > I want to start a discussion thread on KIP-646
> > <https://cwiki.apache.org/confluence/x/RiR4CQ>, which proposes enabling
> > the Serializer API to support ByteBuffer.
> >
> > Please take a look and let me know what you think.
> > I would appreciate any suggestions and feedback.
> >
> > Best regards,
> > TengYao
> >
> >
> >
> 

Reply via email to