Table of Contents |
---|
Status
Current state: "Under Discussion" Accepted
Discussion thread: https://lists.apache.org/thread/5fmgl51x7h3gsx9mn7xq0n28j700rys0
Voting thread:https://lists.apache.org/thread/jgtzlb4l60hs6t3o67x53w8gonk2ffvo
PR: https://github.com/apache/kafka/pull/12685
...
Additionally, I plan to ultimately replace byte[] with ByteBuffer in Serializer: in 3.x versions, both Serializer#serializeToByteBuffer() and Serializer#serialize() exist. Starting from version 4.0, Serializer#serializeToByteBuffer() will be removed and its implementation will replace Serializer#serialize(), which will return ByteBuffer by default, and Serializer#serialize() will not modify the input ByteBuffer.
Public Interfaces
We propose adding default method Serializer#serializeToByteBuffer(String, T), Serializer#serializeToByteBuffer(String, Headers, T) and Partitioner#partition(String, Object, ByteBuffer, Object, ByteBuffer, Cluster):
Class/Interface | Method |
---|---|
Serializer | default ByteBuffer serializeToByteBuffer(String topic, T data) { default ByteBuffer serializeToByteBuffer(String topic, Headers headers, T data) { |
ByteBufferSerializer | /** // Consider that ByteBuffer#wrap(byte[]) return a ByteBuffer that does not need to call flip(). @Override |
Partitioner | default int partition(String topic, Object key, ByteBuffer keyBytes, Object value, ByteBuffer valueBytes, Cluster cluster) { |
...