Thank you for your comment, Divij.

4.  I have added JavaDoc for
`ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)` in the
'Public Interfaces' section, PTAL.

Best,
ShunKang

Divij Vaidya <divijvaidy...@gmail.com> 于2023年4月28日周五 21:06写道:

> Thank you. I have a small nit otherwise the proposal looks good to me.
>
> 4. nit -> please add javadoc in the KIP as well since we are voting on KIP
> in this discussion thread and not on the PR.
>
> --
> Divij Vaidya
>
>
>
> On Fri, Apr 28, 2023 at 1:28 PM ShunKang Lin <linshunkang....@gmail.com>
> wrote:
>
> > Thank you for your comment, Divij.
> >
> >
> > 4. I have added JavaDoc for
> > `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)`, commit
> > was here:
> >
> >
> https://github.com/apache/kafka/pull/12685/commits/a57e0a31c10a5ba49bc2e735b389188e6e071f62
> > ,
> > PTAL.
> >
> > 6. I have added this description in the third paragraph of the Motivation
> > section, PTAL.
> >
> > 7. I mentioned this in the second paragraph of the Motivation section:
> "If
> > T is an instance of ByteBuffer or T is based on ByteBuffer, we would
> reduce
> > a lot of memory allocation and memory copying".
> >
> > Best,
> > ShunKang
> >
> > Divij Vaidya <divijvaidy...@gmail.com> 于2023年4月25日周二 21:11写道:
> >
> > > 4. I did not observe any changes made to the KIP about this. Could we
> > > please add a JavaDoc to serializeToByteBuffer() where we clearly call
> out
> > > that the indices of input ByteBuffer will be modified.
> > >
> > > 5. Thank you. My comment on this is resolved now.
> > >
> > > 6. We should probably mention the strategy to
> > > reconcile serializeToByteBuffer() and serialize() in the KIP. It may
> look
> > > like, in 3.x versions, both exist and starting 4.0,
> > serializeToByteBuffer()
> > > is removed and it replaces the implementation of serialize() which will
> > > return ByteBuffer by default.
> > >
> > > 7. I agree with what Ismael mentioned. We need to clearly mention in
> the
> > > motivation section about the cases which would be optimized. We are
> > > reducing array allocation and data copy. But we are reducing this only
> in
> > > cases where the input ByteBuffer does not satisfy the condition
> > > (data.arrayOffset() == 0 && arr.length == data.remaining()).
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Wed, Apr 19, 2023 at 6:24 AM ShunKang Lin <
> linshunkang....@gmail.com>
> > > wrote:
> > >
> > > > Thank you for your comment, Divij.
> > > >
> > > > 4. Do you still have any questions about #4?
> > > >
> > > > 5. I add test case for ByteBufferSerializer backward compatibility:
> > > >
> > > >
> > >
> >
> https://github.com/apache/kafka/pull/12685/commits/393af38c27ec8d810a2326ac4b89a53b177e3ee1
> > > >
> > > > Best,
> > > > ShunKang
> > > >
> > > > Divij Vaidya <divijvaidy...@gmail.com> 于2023年4月19日周三 00:45写道:
> > > >
> > > > > 3. Ok. Seems like there is no way around to enforce better
> semantics
> > > and
> > > > > maintain backward compatibility as well! Let's go ahead with what
> you
> > > > > proposed and create a JIRA to fix the semantics in version 4.x. My
> > > > comment
> > > > > is resolved here.
> > > > >
> > > > > --
> > > > > Divij Vaidya
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Apr 10, 2023 at 6:47 AM ShunKang Lin <
> > > linshunkang....@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Thanks for your comment.
> > > > > >
> > > > > > This KIP does not modify ByteBufferSerializer#serialize(), so do
> we
> > > > need
> > > > > to
> > > > > > clarify this aspect on motivation?
> > > > > >
> > > > > > Best,
> > > > > > ShunKang
> > > > > >
> > > > > > Ismael Juma <ism...@juma.me.uk>于2023年4月10日 周一12:37写道:
> > > > > >
> > > > > > > Hi,
> > > > > > >
> > > > > > > One interesting aspect is that the current
> `ByteBufferSerializer`
> > > > > avoids
> > > > > > > copies in the following case:
> > > > > > >
> > > > > > > if (data.hasArray()) {
> > > > > > > final byte[] arr = data.array();
> > > > > > > if (data.arrayOffset() == 0 && arr.length == data.remaining())
> {
> > > > > > > return arr;
> > > > > > > }
> > > > > > > }
> > > > > > >
> > > > > > > It would be good to clarify this aspect in the motivation. What
> > > kind
> > > > of
> > > > > > > copies would we avoid (eg direct byte buffers, byte buffer
> views,
> > > > > etc.).
> > > > > > >
> > > > > > > Ismael
> > > > > > >
> > > > > > > On Sun, Sep 25, 2022 at 8:59 AM ShunKang Lin <
> > > > > linshunkang....@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hi all, I'd like to start a new discussion thread on KIP-872
> > > (Kafka
> > > > > > > Client)
> > > > > > > > which proposes that add Serializer#serializeToByteBuffer() to
> > > > reduce
> > > > > > > memory
> > > > > > > > copying.
> > > > > > > >
> > > > > > > > KIP:
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
> > > > > > > > Thanks, ShunKang
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to