Thanks ShunKang, that's a good point about ByteBuffer::hasArray. LGTM!

On Wed, Oct 12, 2022 at 11:12 AM ShunKang Lin <linshunkang....@gmail.com>
wrote:

> Hi Chris,
>
> 1. Record keys/values are duplicated from `DefaultRecordBatch#buffer`, so
> modifying key/value offsets will not modify the original ByteBuffer
> offsets. A read-only ByteBuffer calls `ByteBuffer#hasArray()` to return
> false, which means that a read-only ByteBuffer does not expose the
> underlying array, which is safer but slower when using the ByteBuffer API.
>
> 2. Good idea, I modified the KIP compatibility section, please take a look.
>
> Best,
> ShunKang
>
> Chris Egerton <chr...@aiven.io.invalid> 于2022年10月11日周二 23:59写道:
>
> > Hi ShunKang,
> >
> > Thanks for the KIP! I have a couple thoughts:
> >
> > 1. If we pass the ByteBuffer that we're using internally for the record
> > key/value to the deserializer, it may be mutated by writing new bytes or
> > altering the position. Should we permit this, or would it make sense to
> > provide deserializers with a read-only ByteBuffer? [1]
> >
> > 2. The compatibility section should probably be fleshed out a bit further
> > to state that deserializers that wish to be compatible with older
> versions
> > of the Kafka clients library should always implement the byte array-based
> > deserialize method. We should probably also add this information to the
> > Javadocs for the new method, although this can be taken care of during PR
> > review and doesn't have to be included in the KIP itself.
> >
> > Cheers,
> >
> > Chris
> >
> > [1] -
> >
> >
> https://docs.oracle.com/javase/8/docs/api/java/nio/ByteBuffer.html#asReadOnlyBuffer--
> >
> > On Tue, Oct 11, 2022 at 8:36 AM Luke Chen <show...@gmail.com> wrote:
> >
> > > Hi ShunKang,
> > >
> > > Had a quick look, I think It's a good idea.
> > > I'll check it again tomorrow, and let you know if I have any questions.
> > >
> > > Luke
> > >
> > > On Sun, Sep 25, 2022 at 3:35 PM ShunKang Lin <
> linshunkang....@gmail.com>
> > > wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > > When I try add method `default ByteBuffer
> serializeToByteBuffer(String
> > > > topic, Headers headers, T data)` for `ByteBufferSerializer`, I found
> > > > `ByteBufferSerializer#serialize(String, ByteBuffer)` is not correct.
> > > > Then I searched JIRA and found this:
> > > > https://issues.apache.org/jira/browse/KAFKA-4852, I made a comment
> > below
> > > > this JIRA, PTAL.
> > > >
> > > > Best,
> > > > ShunKang
> > > >
> > > > Guozhang Wang <wangg...@gmail.com> 于2022年9月20日周二 06:33写道:
> > > >
> > > > > A separate question regarding the proposed API as well: what do you
> > > think
> > > > > to also augment the serializers with ByteBuffer return type in
> order
> > to
> > > > be
> > > > > symmetric with deserializers?
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Sep 19, 2022 at 3:32 PM Guozhang Wang <wangg...@gmail.com>
> > > > wrote:
> > > > >
> > > > > > Hello ShunKang,
> > > > > >
> > > > > > Thanks for filing the proposal, and sorry for the late reply!
> > > > > >
> > > > > > I looked over your KIP proposal and the PR, in general I think I
> > > agree
> > > > > > that adding an overloaded function with `ByteBuffer` param is
> > > > beneficial,
> > > > > > but I have a meta question regarding it's impact on Kafka
> consumer:
> > > my
> > > > > > understanding from your PR is that, we can only save memory
> > > allocations
> > > > > if
> > > > > > the key/value types happen to be ByteBuffer as well, otherwise we
> > > would
> > > > > > still do the `return deserialize(topic, headers,
> > > Utils.toArray(data));`
> > > > > > from default impls unless the user customized deserializers is
> > > > augmented
> > > > > to
> > > > > > handle ByteBuffer directly, right?
> > > > > >
> > > > > >
> > > > > > Guozhang
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Sun, Aug 21, 2022 at 9:56 AM ShunKang Lin <
> > > > linshunkang....@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > >> Hi all,
> > > > > >>
> > > > > >> I'd like to start a discussion on KIP-863 which is Reduce
> > > > > >> Fetcher#parseRecord() memory copy. This KIP can reduce Kafka
> > > Consumer
> > > > > >> memory allocation by nearly 50% during fetch records.
> > > > > >>
> > > > > >> Please check
> > > > > >>
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035
> > > > > >> and https://github.com/apache/kafka/pull/12545 for more
> details.
> > > > > >>
> > > > > >> Any feedbacks and comments are welcomed.
> > > > > >>
> > > > > >> Thanks.
> > > > > >>
> > > > > >
> > > > > >
> > > > > > --
> > > > > > -- Guozhang
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>

Reply via email to