Re: Apache Kafka 3.6.0 release

2023-07-22 Thread ShunKang Lin
Hi Satish,

Could we add "KIP-863: Reduce CompletedFetch#parseRecord() memory copy" [1]
and "KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory
copying" [2] to the release plan?
Thanks!

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035

[2]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
I would appreciate a few more reviews on the pull request (
https://github.com/apache/kafka/pull/12685) for KIP-872.

Best,
ShunKang

Divij Vaidya  于2023年7月22日周六 20:06写道:

> Hi Satish
>
> I have added the following accepted KIPs to the release plan. Please let me
> know if something requires a change.
>
> Accepted KIPs -
>
> 1.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-852%3A+Optimize+calculation+of+size+for+log+in+remote+tier
>
> 2.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-937%3A+Improve+Message+Timestamp+Validation
>
>
> Pending discussion KIP which I believe is important to be merged into 3.6 -
>
> 3.
>
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
>
>
> --
> Divij Vaidya
>
>
>
> On Sat, Jul 22, 2023 at 6:41 AM Satish Duggana 
> wrote:
>
> > Thanks Hao for the update on KIP-925.
> >
> > On Thu, 20 Jul 2023 at 23:05, Hao Li  wrote:
> > >
> > > Hi Satish,
> > >
> > > KIP-925 was accepted and currently under implementation. I just added
> it
> > to
> > > the release plan.
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-925%3A+Rack+aware+task+assignment+in+Kafka+Streams
> > >
> > > Thanks,
> > > Hao
> > >
> > > On Thu, Jul 20, 2023 at 6:18 AM Christo Lolov 
> > > wrote:
> > >
> > > > Hello!
> > > >
> > > > A couple of days ago I opened a new KIP for discussion - KIP-952
> [1]. I
> > > > believe it might be a blocker for the release of 3.6.0, but I wanted
> to
> > > > bring it up here for a decision on its urgency with the current set
> of
> > > > people who are looking at Tiered Storage (Satish, Luke, Ivan, Divij)
> > given
> > > > that the date for KIP freeze is fast approaching.
> > > > What are your thoughts on the matter?
> > > >
> > > > [1]
> > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-952%3A+Regenerate+segment-aligned+producer+snapshots+when+upgrading+to+a+Kafka+version+supporting+Tiered+Storage
> > > >
> > > > Best,
> > > > Christo
> > > >
> > > > On Sat, 8 Jul 2023 at 13:06, Satish Duggana <
> satish.dugg...@gmail.com>
> > > > wrote:
> > > >
> > > > > Hi Yash,
> > > > > Thanks for the update. Added KIP-793 to the release plan. Please
> feel
> > > > > free to update the release wiki with any other updates on the KIP.
> > > > >
> > > > > ~Satish.
> > > > >
> > > > > On Fri, 7 Jul 2023 at 10:52, Yash Mayya 
> > wrote:
> > > > > >
> > > > > > Hi Satish,
> > > > > >
> > > > > > KIP-793 [1] just passed voting and we should be able to wrap up
> the
> > > > > > implementation in time for the 3.6.0 feature freeze. Could we add
> > it to
> > > > > the
> > > > > > release plan?
> > > > > >
> > > > > > [1] -
> > > > > >
> > > > >
> > > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-793%3A+Allow+sink+connectors+to+be+used+with+topic-mutating+SMTs
> > > > > >
> > > > > > Thanks,
> > > > > > Yash
> > > > > >
> > > > > > On Mon, Jun 12, 2023 at 3:52 PM Satish Duggana <
> > > > satish.dugg...@gmail.com
> > > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hi,
> > > > > > > I have created a release plan for Apache Kafka version 3.6.0 on
> > the
> > > > > > > wiki. You can access the release plan and all related
> > information by
> > > > > > > following this link:
> > > > > > >
> > https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+3.6.0
> > > > > > >
> > > > > > > The release plan outlines the key milestones and important
> dates
> > for
> > > > > > > version 3.6.0. Currently, the following dates have been set for
> > the
> > > > > > > release:
> > > > > > >
> > > > > > > KIP Freeze: 26th July 23
> > > > > > > Feature Freeze : 16th Aug 23
> > > > > > > Code Freeze : 30th Aug 23
> > > > > > >
> > > > > > > Please review the release plan and provide any additional
> > information
> > > > > > > or updates regarding KIPs targeting version 3.6.0. If you have
> > > > > > > authored any KIPs that are missing a status or if there are
> > incorrect
> > > > > > > status details, please make the necessary updates and inform me
> > so
> > > > > > > that I can keep the plan accurate and up to date.
> > > > > > >
> > > > > > > Thanks,
> > > > > > > Satish.
> > > > > > >
> > > > > > > On Mon, 17 Apr 2023 at 21:17, Luke Chen 
> > wrote:
> > > > > > > >
> > > > > > > > Thanks for volunteering!
> > > > > > > >
> > > > > > > > +1
> > > > > > > >
> > > > > > > > Luke
> > > > > > > >
> > > > > > > > On Mon, Apr 17, 2023 at 2:03 AM Ismael Juma <
> ism...@juma.me.uk
> > >
> > > > > wrote:
> > > > > > > >
> > > > > > > > > Thanks for 

Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-06-20 Thread ShunKang Lin
Hi all,

Thanks all for voting!

Currently we have +3 binding (from Divij Vaidya, Luke Chen and ziming deng)
and +2 non-binding(from Kirk True and Kamal Chandraprakash).

This vote thread has been open over 72 hours and has sufficient votes, so
I’ll close the voting at this time.

KIP-872 has PASSED.

Best,
ShunKang


在 2023年6月21日星期三,Divij Vaidya  写道:

> +1 (again) - binding :)
>
> Please update the PR ShunKang and tag me for review when you are ready.
>
> --
> Divij Vaidya
>
>
>
> On Tue, Jun 20, 2023 at 4:11 PM John Roesler  wrote:
>
> > Hi Divij and ShunKang,
> >
> > I pulled open this thread to see if you needed my vote, but FYI, Divij is
> > a committer now, so he can re-cast his vote as binding.
> >
> > Thanks,
> > -John
> >
> > On 2023/06/20 13:37:04 ShunKang Lin wrote:
> > > Hi all,
> > >
> > > Bump this thread again and see if we could get a few more votes.
> > > Currently we have +3 non-binding (from Divij Vaidya, Kirk True and
> Kamal
> > > Chandraprakash)  and +2 binding (from Luke Chen and ziming deng).
> > > Hoping we can get this approved, reviewed, and merged in time for
> 3.6.0.
> > >
> > > Best,
> > > ShunKang
> > >
> > > ShunKang Lin  于2023年5月7日周日 15:24写道:
> > >
> > > > Hi everyone,
> > > >
> > > > I'd like to open the vote for KIP-872, which proposes to add
> > > > Serializer#serializeToByteBuffer() to reduce memory copying.
> > > >
> > > > The proposal is here:
> > > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
> > > >
> > > > The pull request is here:
> > > > https://github.com/apache/kafka/pull/12685
> > > >
> > > > Thanks to all who reviewed the proposal, and thanks in advance for
> > taking
> > > > the time to vote!
> > > >
> > > > Best,
> > > > ShunKang
> > > >
> > >
> >
>


Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-06-20 Thread ShunKang Lin
Hi John,

Thank you for your reminder!

Best,
ShunKang

John Roesler 于2023年6月20日 周二22:11写道:

> Hi Divij and ShunKang,
>
> I pulled open this thread to see if you needed my vote, but FYI, Divij is
> a committer now, so he can re-cast his vote as binding.
>
> Thanks,
> -John
>
> On 2023/06/20 13:37:04 ShunKang Lin wrote:
> > Hi all,
> >
> > Bump this thread again and see if we could get a few more votes.
> > Currently we have +3 non-binding (from Divij Vaidya, Kirk True and Kamal
> > Chandraprakash)  and +2 binding (from Luke Chen and ziming deng).
> > Hoping we can get this approved, reviewed, and merged in time for 3.6.0.
> >
> > Best,
> > ShunKang
> >
> > ShunKang Lin  于2023年5月7日周日 15:24写道:
> >
> > > Hi everyone,
> > >
> > > I'd like to open the vote for KIP-872, which proposes to add
> > > Serializer#serializeToByteBuffer() to reduce memory copying.
> > >
> > > The proposal is here:
> > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
> > >
> > > The pull request is here:
> > > https://github.com/apache/kafka/pull/12685
> > >
> > > Thanks to all who reviewed the proposal, and thanks in advance for
> taking
> > > the time to vote!
> > >
> > > Best,
> > > ShunKang
> > >
> >
>


Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-06-20 Thread ShunKang Lin
Hi all,

Bump this thread again and see if we could get a few more votes.
Currently we have +3 non-binding (from Divij Vaidya, Kirk True and Kamal
Chandraprakash)  and +2 binding (from Luke Chen and ziming deng).
Hoping we can get this approved, reviewed, and merged in time for 3.6.0.

Best,
ShunKang

ShunKang Lin  于2023年5月7日周日 15:24写道:

> Hi everyone,
>
> I'd like to open the vote for KIP-872, which proposes to add
> Serializer#serializeToByteBuffer() to reduce memory copying.
>
> The proposal is here:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
>
> The pull request is here:
> https://github.com/apache/kafka/pull/12685
>
> Thanks to all who reviewed the proposal, and thanks in advance for taking
> the time to vote!
>
> Best,
> ShunKang
>


Re: [ANNOUNCE] New committer: Divij Vaidya

2023-06-16 Thread ShunKang Lin
Congratulations, Divij!

Bill Bejeck  于2023年6月16日周五 09:43写道:

> Congratulations!!
>
> On Tue, Jun 13, 2023 at 7:14 PM ziming deng 
> wrote:
>
> > Congratulations, Divij!
> > Well deserved!
> >
> > --
> > Ziming
> >
> > > On Jun 14, 2023, at 09:41, Luke Chen  wrote:
> > >
> > > Congratulations, Divij!
> > > Well deserved!
> > >
> > > Luke
> > >
> > > On Wed, Jun 14, 2023 at 7:01 AM Viktor Somogyi-Vass
> > >  wrote:
> > >
> > >> Congrats Divij!
> > >>
> > >> On Tue, Jun 13, 2023, 20:27 Philip Nee  wrote:
> > >>
> > >>> Congrats!
> > >>>
> > >>> On Tue, Jun 13, 2023 at 8:17 PM Randall Hauch 
> > wrote:
> > >>>
> >  Congratulations!
> > 
> >  On Tue, Jun 13, 2023 at 12:48 PM Matthias J. Sax 
> > >>> wrote:
> > 
> > > Congrats!
> > >
> > > On 6/13/23 10:24 AM, Satish Duggana wrote:
> > >> Congratulations Divij!!
> > >>
> > >> On Tue, 13 Jun 2023 at 22:41, Manyanda Chitimbo
> > >>  wrote:
> > >>>
> > >>> Congratulations Divij.
> > >>>
> > >>> On Tue 13 Jun 2023 at 17:50, Bruno Cadonna 
> >  wrote:
> > >>>
> >  Hi all,
> > 
> >  The PMC of Apache Kafka is pleased to announce a new Kafka
> > >>> committer
> >  Divij Vaidya.
> > 
> >  Divij's major contributions are:
> > 
> >  GDPR compliance enforcement of kafka-site -
> >  https://issues.apache.org/jira/browse/KAFKA-13868
> > 
> >  Performance improvements:
> > 
> >  Improve performance of VarInt encoding and decoding -
> >  https://github.com/apache/kafka/pull/13312
> > 
> >  Reduce data copy & buffer allocation during decompression -
> >  https://github.com/apache/kafka/pull/13135
> > 
> >  He also was heavily involved in the migration to Mockito.
> > 
> >  Furthermore, Divij is very active on the mailing lists as well
> as
> > >>> in
> >  maintaining and reviewing pull requests.
> > 
> >  Congratulations, Divij!
> > 
> >  Thanks,
> > 
> >  Bruno (on behalf of the Apache Kafka PMC)
> > 
> > 
> >  --
> > >>> Manyanda Chitimbo.
> > >
> > 
> > >>>
> > >>
> >
> >
>


Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-06-11 Thread ShunKang Lin
Hi all,

Thanks for voting!

Bump this thread again and see if we could get a few more votes. Currently
we have +3 non-binding (from Divij Vaidya, Kirk True and Kamal
Chandraprakash)  and +2 binding (from Luke Chen and ziming deng).
Hoping we can get this approved, reviewed, and merged in time for 3.6.0.

Best,
ShunKang

ShunKang Lin  于2023年5月7日周日 15:24写道:

> Hi everyone,
>
> I'd like to open the vote for KIP-872, which proposes to add
> Serializer#serializeToByteBuffer() to reduce memory copying.
>
> The proposal is here:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
>
> The pull request is here:
> https://github.com/apache/kafka/pull/12685
>
> Thanks to all who reviewed the proposal, and thanks in advance for taking
> the time to vote!
>
> Best,
> ShunKang
>


Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-05-30 Thread ShunKang Lin
Hi all,

Bump this thread again and see if we could get a few more votes. Currently
we have +2 non-binding and +1 binding.
Hoping we can get this approved, reviewed, and merged in time for 3.6.0.

Best,
ShunKang

ShunKang Lin  于2023年5月7日周日 15:24写道:

> Hi everyone,
>
> I'd like to open the vote for KIP-872, which proposes to add
> Serializer#serializeToByteBuffer() to reduce memory copying.
>
> The proposal is here:
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
>
> The pull request is here:
> https://github.com/apache/kafka/pull/12685
>
> Thanks to all who reviewed the proposal, and thanks in advance for taking
> the time to vote!
>
> Best,
> ShunKang
>


Re: [VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-05-21 Thread ShunKang Lin
Hi all,

Thanks for voting!
Just wanted to bump this thread and see if we could get a few more votes.
Currently we have +2 non-binding and +1 binding.
Hoping we can get this approved, reviewed, and merged in time for 3.6.0.

Best,
ShunKang

Kirk True  于2023年5月11日周四 22:54写道:

> +1 (non-binding)
>
> Thanks ShunKang!
>
> On Wed, May 10, 2023, at 7:26 PM, Luke Chen wrote:
> > +1(binding) from me.
> > Thanks for the improvement!
> >
> > Luke
> >
> > On Sun, May 7, 2023 at 6:34 PM Divij Vaidya 
> wrote:
> >
> > > Vote +1 (non binding)
> > >
> > > I think that this is a nice improvement as it prevents an unnecessary
> data
> > > copy for users who are using ByteBuffer serialization on the producer.
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Sun, May 7, 2023 at 9:24 AM ShunKang Lin  >
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I'd like to open the vote for KIP-872, which proposes to add
> > > > Serializer#serializeToByteBuffer() to reduce memory copying.
> > > >
> > > > The proposal is here:
> > > >
> > >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828
> > > >
> > > > The pull request is here:
> > > > https://github.com/apache/kafka/pull/12685
> > > >
> > > > Thanks to all who reviewed the proposal, and thanks in advance for
> taking
> > > > the time to vote!
> > > >
> > > > Best,
> > > > ShunKang
> > > >
> > >
> >
>


[VOTE] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-05-07 Thread ShunKang Lin
Hi everyone,

I'd like to open the vote for KIP-872, which proposes to add
Serializer#serializeToByteBuffer() to reduce memory copying.

The proposal is here:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495828

The pull request is here:
https://github.com/apache/kafka/pull/12685

Thanks to all who reviewed the proposal, and thanks in advance for taking
the time to vote!

Best,
ShunKang


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-04-28 Thread ShunKang Lin
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  于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 
> 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  于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  于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 于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
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-04-28 Thread ShunKang Lin
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  于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 
> 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  于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 于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
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-04-18 Thread ShunKang Lin
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  于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 
> 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 于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
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-04-09 Thread ShunKang Lin
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 于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 
> 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
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2023-04-09 Thread ShunKang Lin
Sorry for the late reply due to recent work overload.

3. If the ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)
method does not maintain the same behavior as the
ByteBufferSerializer#serialize(String, ByteBuffer) method, it will break
the code logic for those users who originally used ByteBufferSerializer to
serialize messages. Please refer to PR for KIP-872:
https://github.com/apache/kafka/pull/12685/files#diff-42d8f5166459ee28f201ff9cec0080fc7845544a0089ac9e8f3e16864cc1193eR1006
and
https://github.com/apache/kafka/pull/12685/files#diff-42d8f5166459ee28f201ff9cec0080fc7845544a0089ac9e8f3e16864cc1193eR1015
for details.

4. Changes to the ByteBufferSerializer documentation should be made only
after a final decision is reached on #3.

5. The same applies to #5, and we should wait for a final decision on #3.

Best,
ShunKang

Divij Vaidya  于2022年11月7日周一 21:12写道:

> Apologies for the late reply. I will be more proactive on this thead moving
> ahead.
>
> 3. Understood. Perhaps, `ByteBuffer#asReadOnlyBuffer()` is not the right
> solution and I acknowledge that the current API modifies the offsets of the
> user provided input when it calls flip(). I still believe that we should
> not be modifying (both offsets and data) the user provided input
> bytebuffer, but I understand that it would require a change in semantics
> wrt existing API behaviour. I would vote for having new/correct semantics
> introduced with this KIP itself (else, as John mentioned, we would have to
> add a new method later). I am advocating for new semantics because it
> clarifies the contract rigorously (immutable state of input params) which
> enables consumers of the API to do some nifty things on their end wrt
> memory management.
>
> 4. Regarding #4 I mentioned earlier, do you agree with the comment? If yes,
> can you please add the JavaDocs to the API contract defined in KIP?
>
> 5. From a backward compatibility perspective, would the offsets for the
> user provided ByteBuffer (key & value) remain the same as the earlier
> implementation for `doSend()`? Could we add a test to verify this? Perhaps,
> this is worth clarifying in the KIP?
>
> --
> Divij Vaidya
>
>
>
> On Sun, Nov 6, 2022 at 4:23 PM John Roesler  wrote:
>
> > Thanks for the reply, ShunKang!
> >
> > You’re absolutely right, we should not change the behavior of the
> existing
> > method.
> >
> > Regarding the new method, I was thinking that this is a good opportunity
> > to correct what seems to be strange semantics in the original one. If we
> > keep the same semantics and want to correct it later, we’ll be forced to
> > introduce yet another method later. This especially makes sense if we’re
> > thinking of deprecating the original method. But if you think it’s better
> > to keep it the way it is, I’m fine with it.
> >
> > I have no other comments.
> >
> > Thanks again for the KIP,
> > John
> >
> > On Sat, Nov 5, 2022, at 11:59, ShunKang Lin wrote:
> > > Hi John,
> > >
> > > Thanks for your comments!
> > >
> > > For your first question, I see some unit test cases that give us a
> > > ByteBuffer not set to read before calling
> > > `ByteBufferSerializer#serialize(String, ByteBuffer)`, e.g.
> > > `ArticleSerializer`, `AugmentedArticleSerializer`,
> > > `AugmentedCommentSerializer` and `CommentSerializer`. If we don't flip
> > the
> > > ByteBuffer inside the `ByteBufferSerializer#serialize(String,
> > ByteBuffer)`
> > > it will break user code using `ByteBufferSerializer#serialize(String,
> > > ByteBuffer)`, and if we don't flip the ByteBuffer inside
> > > the `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)`,
> it
> > > will be even more strange to the user, because
> > > `ByteBufferSerializer#serialize(String, ByteBuffer)` and
> > > `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)`
> require
> > > users use the ByteBufferSerializer in two different ways. So if we
> think
> > of
> > > `ByteBufferSerialize#serializeToByteBuffer(String, ByteBuffer)` as
> > setting
> > > up a ByteBuffer to read later, is it more acceptable?
> > >
> > > For your second question, I plan to ultimately replace byte[] with
> > > ByteBuffer, I will document the intent in your KIP and JavaDocs later.
> > >
> > > I will clarify that if a Serializer implements the new method, then the
> > old
> > > one will never be called.
> > >
> > > Best,
> > > ShunKang
> > >
> > > John Roesler  于2022年11月4日周五 22:42写道:
> > >
> > >> Hi ShunKang,
> 

Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-11-08 Thread ShunKang Lin
Thanks John!

I will consider your opinion, I’ve been busy at work lately.

Best,
ShunKang

John Roesler 于2022年11月6日 周日23:22写道:

> Thanks for the reply, ShunKang!
>
> You’re absolutely right, we should not change the behavior of the existing
> method.
>
> Regarding the new method, I was thinking that this is a good opportunity
> to correct what seems to be strange semantics in the original one. If we
> keep the same semantics and want to correct it later, we’ll be forced to
> introduce yet another method later. This especially makes sense if we’re
> thinking of deprecating the original method. But if you think it’s better
> to keep it the way it is, I’m fine with it.
>
> I have no other comments.
>
> Thanks again for the KIP,
> John
>
> On Sat, Nov 5, 2022, at 11:59, ShunKang Lin wrote:
> > Hi John,
> >
> > Thanks for your comments!
> >
> > For your first question, I see some unit test cases that give us a
> > ByteBuffer not set to read before calling
> > `ByteBufferSerializer#serialize(String, ByteBuffer)`, e.g.
> > `ArticleSerializer`, `AugmentedArticleSerializer`,
> > `AugmentedCommentSerializer` and `CommentSerializer`. If we don't flip
> the
> > ByteBuffer inside the `ByteBufferSerializer#serialize(String,
> ByteBuffer)`
> > it will break user code using `ByteBufferSerializer#serialize(String,
> > ByteBuffer)`, and if we don't flip the ByteBuffer inside
> > the `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)`, it
> > will be even more strange to the user, because
> > `ByteBufferSerializer#serialize(String, ByteBuffer)` and
> > `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)` require
> > users use the ByteBufferSerializer in two different ways. So if we think
> of
> > `ByteBufferSerialize#serializeToByteBuffer(String, ByteBuffer)` as
> setting
> > up a ByteBuffer to read later, is it more acceptable?
> >
> > For your second question, I plan to ultimately replace byte[] with
> > ByteBuffer, I will document the intent in your KIP and JavaDocs later.
> >
> > I will clarify that if a Serializer implements the new method, then the
> old
> > one will never be called.
> >
> > Best,
> > ShunKang
> >
> > John Roesler  于2022年11月4日周五 22:42写道:
> >
> >> Hi ShunKang,
> >>
> >> Thanks for the KIP!
> >>
> >> I’ve been wanting to transition toward byte buffers for a while, so this
> >> is a nice start.
> >>
> >> I thought it was a bit weird to flip the buffer inside the serializer,
> but
> >> I see the existing one already does that. I would have thought it would
> >> make more sense for the caller to give us a buffer already set up for
> >> reading. Do you think it makes sense to adopt this pattern for the new
> >> method?
> >>
> >> Do you plan to keep the new methods as optional indefinitely, or do you
> >> plan to ultimately replace byte[] with ByteBuffer? If it’s the latter,
> then
> >> it would be good to document the intent in your KIP and JavaDocs.
> >>
> >> It would be good to clarify that if a Serializer implements the new
> >> method, then the old one will never be called. That way, implementations
> >> can just throw an exception on that method instead of implementing both.
> >>
> >> Thanks again!
> >> -John
> >>
> >> On Wed, Nov 2, 2022, at 20:14, ShunKang Lin wrote:
> >> > Bump this thread again : )
> >> >
> >> > ShunKang Lin 于2022年9月25日 周日23:59写道:
> >> >
> >> >> 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
> >> >>
> >>
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-11-05 Thread ShunKang Lin
Hi John,

Thanks for your comments!

For your first question, I see some unit test cases that give us a
ByteBuffer not set to read before calling
`ByteBufferSerializer#serialize(String, ByteBuffer)`, e.g.
`ArticleSerializer`, `AugmentedArticleSerializer`,
`AugmentedCommentSerializer` and `CommentSerializer`. If we don't flip the
ByteBuffer inside the `ByteBufferSerializer#serialize(String, ByteBuffer)`
it will break user code using `ByteBufferSerializer#serialize(String,
ByteBuffer)`, and if we don't flip the ByteBuffer inside
the `ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)`, it
will be even more strange to the user, because
`ByteBufferSerializer#serialize(String, ByteBuffer)` and
`ByteBufferSerializer#serializeToByteBuffer(String, ByteBuffer)` require
users use the ByteBufferSerializer in two different ways. So if we think of
`ByteBufferSerialize#serializeToByteBuffer(String, ByteBuffer)` as setting
up a ByteBuffer to read later, is it more acceptable?

For your second question, I plan to ultimately replace byte[] with
ByteBuffer, I will document the intent in your KIP and JavaDocs later.

I will clarify that if a Serializer implements the new method, then the old
one will never be called.

Best,
ShunKang

John Roesler  于2022年11月4日周五 22:42写道:

> Hi ShunKang,
>
> Thanks for the KIP!
>
> I’ve been wanting to transition toward byte buffers for a while, so this
> is a nice start.
>
> I thought it was a bit weird to flip the buffer inside the serializer, but
> I see the existing one already does that. I would have thought it would
> make more sense for the caller to give us a buffer already set up for
> reading. Do you think it makes sense to adopt this pattern for the new
> method?
>
> Do you plan to keep the new methods as optional indefinitely, or do you
> plan to ultimately replace byte[] with ByteBuffer? If it’s the latter, then
> it would be good to document the intent in your KIP and JavaDocs.
>
> It would be good to clarify that if a Serializer implements the new
> method, then the old one will never be called. That way, implementations
> can just throw an exception on that method instead of implementing both.
>
> Thanks again!
> -John
>
> On Wed, Nov 2, 2022, at 20:14, ShunKang Lin wrote:
> > Bump this thread again : )
> >
> > ShunKang Lin 于2022年9月25日 周日23:59写道:
> >
> >> 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
> >>
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-11-02 Thread ShunKang Lin
Bump this thread again : )

ShunKang Lin 于2022年9月25日 周日23:59写道:

> 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
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-10-28 Thread ShunKang Lin
Bump this thread to see if there are any comments/thoughts.

Best,
ShunKang


ShunKang Lin  于2022年9月30日周五 23:58写道:

> Hi Divij Vaidya,
>
> 3. Sounds good, but `ByteBuffer#asReadOnlyBuffer()` returns a read-only
> `ByteBuffer` which `ByteBuffer#hasArray()` returns false, then it will make
> `Utils#writeTo(DataOutput, ByteBuffer, int)` perform efficiently Lower
> (called in `DefaultRecord#writeTo(DataOutputStream, int, long, ByteBuffer,
> ByteBuffer, Header[])`). By the way,
> `ByteBufferSerializer#serialize(String, ByteBuffer)` has called
> `ByteBuffer#flip()` which will modify the offset of the input buffer.
>
> In my opinion, it is acceptable to modify the offset of the input buffer.
> After all, serialization means reading data and `ByteBuffer` needs to
> modify the position and limit before reading the data. We just need to
> assure the user that the input data will not be modified by the Kafka
> library.
>
> Divij Vaidya  于2022年9月29日周四 19:07写道:
>
>> 1. You are right. We append the message to the `DefaultRecord` and append
>> is a copy operation. Hence, the ByteBuffer would be released at the end of
>> the KafkaProducer#doSend() method. This comment is resolved.
>> 2. I don't foresee any compatibility issues since #1 is not a problem
>> anymore. This comment is resolved.
>>
>> New comments:
>>
>> 3. In the ByteBufferSerializer#serializeToByteBuffer, could we take the
>> input ByteBuffer from the user application and return a
>> `data.asReadOnlyBuffer()`? As I understand, it does not involve any data
>> copy, hence no extra memory cost. On the upside, it would help provide the
>> guarantee to the user that the data (and the points such as position, cap
>> etc.) in the input ByteBuffer is not modified by the Kafka library.
>>
>> 4. Please change the documentation of the ByteBufferSerializer to clarify
>> that Kafka code will not modify the buffer (neither the data of the
>> provided input buffer nor the pointers).
>>
>> --
>> Divij Vaidya
>>
>>
>>
>> On Wed, Sep 28, 2022 at 5:35 PM ShunKang Lin 
>> wrote:
>>
>> > Hi Divij Vaidya,
>> >
>> > Thanks for your comments.
>> >
>> > 1. I checked the code of KafkaProducer#doSend()
>> > and RecordAccumulator#append(), if KafkaProducer#doSend() returns it
>> means
>> > serializedKey and serializedValue have been appended to
>> > ProducerBatch#recordsBuilder and we don't keep reference of
>> serializedKey
>> > and serializedValue.
>> >
>> > 2. According to 1, the user application can reuse the ByteBuffer to send
>> > consecutive KafkaProducer#send() requests without breaking the user
>> > application. If we are concerned about compatibility, we can provide
>> > another Serializer, such as ZeroCopyByteBufferSerializer, and keep the
>> > original ByteBufferSerializer unchanged.
>> >
>> > In my opinion, kafka-clients should provide some way for users who want
>> to
>> > improve application performance, if users want to improve application
>> > performance, they should use lower level code and understand the
>> underlying
>> > implementation of these codes.
>> >
>> > Best,
>> > ShunKang
>> >
>> > Divij Vaidya  于2022年9月28日周三 19:58写道:
>> >
>> > > Hello
>> > >
>> > > I believe that the current behaviour of creating a copy of the user
>> > > provided input is the correct approach because of the following
>> reasons:
>> > >
>> > > 1. In the existing implementation (considering cases when T is
>> ByteBuffer
>> > > in Serializer#serialize(String,Headers,T)) we copy the data (T) into a
>> > new
>> > > byte[]. In the new approach, we would continue to re-use the
>> ByteBuffer
>> > > even after doSend() which means the `ProducerRecord` object cannot go
>> out
>> > > of scope from a GC perspective at the end of doSend(). Hence, the new
>> > > approach may lead to increased heap memory usage for a greater period
>> of
>> > > time.
>> > >
>> > > 2. The new approach may break certain user applications e.g. consider
>> an
>> > > user application which re-uses the ByteBuffer (maybe it's a memory
>> mapped
>> > > byte buffer) to send consecutive Producer.send() requests. Prior to
>> this
>> > > change, they could do that because we copy the data from user provided
>> > > input before storing it in the accumulator but after this change, they
>>

Re: [VOTE] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-10-24 Thread ShunKang Lin
Hi everyone,

Thanks Luke.
I have submitted a PR  https://github.com/apache/kafka/pull/12545, please
take a look.

Best,
ShunKang

Luke Chen  于2022年10月24日周一 09:56写道:

> Hi ShunKang,
> Yes, this vote can be concluded.
> You can start to implement it.
>
> Thanks.
> Luke
>
>
> On Fri, Oct 21, 2022 at 9:45 PM ShunKang Lin 
> wrote:
>
> > Hi everyone,
> >
> > Thank you for the vote. I've got three +1 votes (Guozhang, Luke, Chris),
> > can this vote be concluded?
> >
> > Best,
> > ShunKang
> >
> > Chris Egerton  于2022年10月12日周三 23:17写道:
> >
> > > +1 (binding)
> > > Thanks ShunKang!
> > >
> > > On Tue, Oct 11, 2022 at 9:26 PM Luke Chen  wrote:
> > >
> > > > +1 from me.
> > > > Thanks for the KIP.
> > > >
> > > > Luke
> > > >
> > > > On Fri, Sep 23, 2022 at 1:50 AM Guozhang Wang 
> > > wrote:
> > > >
> > > > > +1, thanks ShunKang.
> > > > >
> > > > > Though its proposed motivation is on consumer fetcher's
> > > deserialization,
> > > > I
> > > > > think adding an overloaded method with ByteBuffer would help with
> > other
> > > > > serde places on the client side as well.
> > > > >
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Thu, Sep 22, 2022 at 9:41 AM ShunKang Lin <
> > > linshunkang@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Hi everyone,
> > > > > >
> > > > > > I'd like to open the vote for KIP-863, which proposes to reduce
> > > memory
> > > > > > allocation and memory copying in
> > Fetcher#parseRecord(TopicPartition,
> > > > > > RecordBatch, Record).
> > > > > >
> > > > > > The proposal is here:
> > > > > >
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035
> > > > > >
> > > > > > Thanks to all who reviewed the proposal, and thanks in advance
> for
> > > > taking
> > > > > > the time to vote!
> > > > > >
> > > > > > Best,
> > > > > > ShunKang
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> >
>


Re: [VOTE] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-10-21 Thread ShunKang Lin
Hi everyone,

Thank you for the vote. I've got three +1 votes (Guozhang, Luke, Chris),
can this vote be concluded?

Best,
ShunKang

Chris Egerton  于2022年10月12日周三 23:17写道:

> +1 (binding)
> Thanks ShunKang!
>
> On Tue, Oct 11, 2022 at 9:26 PM Luke Chen  wrote:
>
> > +1 from me.
> > Thanks for the KIP.
> >
> > Luke
> >
> > On Fri, Sep 23, 2022 at 1:50 AM Guozhang Wang 
> wrote:
> >
> > > +1, thanks ShunKang.
> > >
> > > Though its proposed motivation is on consumer fetcher's
> deserialization,
> > I
> > > think adding an overloaded method with ByteBuffer would help with other
> > > serde places on the client side as well.
> > >
> > >
> > > Guozhang
> > >
> > > On Thu, Sep 22, 2022 at 9:41 AM ShunKang Lin <
> linshunkang@gmail.com>
> > > wrote:
> > >
> > > > Hi everyone,
> > > >
> > > > I'd like to open the vote for KIP-863, which proposes to reduce
> memory
> > > > allocation and memory copying in Fetcher#parseRecord(TopicPartition,
> > > > RecordBatch, Record).
> > > >
> > > > The proposal is here:
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035
> > > >
> > > > Thanks to all who reviewed the proposal, and thanks in advance for
> > taking
> > > > the time to vote!
> > > >
> > > > Best,
> > > > ShunKang
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-10-12 Thread ShunKang Lin
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  于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  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 
> > 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  于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 
> > > 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
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-10-12 Thread ShunKang Lin
Hi Luke,

Thanks for your comments.

Best,
ShunKang

Luke Chen  于2022年10月11日周二 20:36写道:

> 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 
> 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  于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 
> > 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
> > >
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-09-30 Thread ShunKang Lin
Hi Divij Vaidya,

3. Sounds good, but `ByteBuffer#asReadOnlyBuffer()` returns a read-only
`ByteBuffer` which `ByteBuffer#hasArray()` returns false, then it will make
`Utils#writeTo(DataOutput, ByteBuffer, int)` perform efficiently Lower
(called in `DefaultRecord#writeTo(DataOutputStream, int, long, ByteBuffer,
ByteBuffer, Header[])`). By the way,
`ByteBufferSerializer#serialize(String, ByteBuffer)` has called
`ByteBuffer#flip()` which will modify the offset of the input buffer.

In my opinion, it is acceptable to modify the offset of the input buffer.
After all, serialization means reading data and `ByteBuffer` needs to
modify the position and limit before reading the data. We just need to
assure the user that the input data will not be modified by the Kafka
library.

Divij Vaidya  于2022年9月29日周四 19:07写道:

> 1. You are right. We append the message to the `DefaultRecord` and append
> is a copy operation. Hence, the ByteBuffer would be released at the end of
> the KafkaProducer#doSend() method. This comment is resolved.
> 2. I don't foresee any compatibility issues since #1 is not a problem
> anymore. This comment is resolved.
>
> New comments:
>
> 3. In the ByteBufferSerializer#serializeToByteBuffer, could we take the
> input ByteBuffer from the user application and return a
> `data.asReadOnlyBuffer()`? As I understand, it does not involve any data
> copy, hence no extra memory cost. On the upside, it would help provide the
> guarantee to the user that the data (and the points such as position, cap
> etc.) in the input ByteBuffer is not modified by the Kafka library.
>
> 4. Please change the documentation of the ByteBufferSerializer to clarify
> that Kafka code will not modify the buffer (neither the data of the
> provided input buffer nor the pointers).
>
> --
> Divij Vaidya
>
>
>
> On Wed, Sep 28, 2022 at 5:35 PM ShunKang Lin 
> wrote:
>
> > Hi Divij Vaidya,
> >
> > Thanks for your comments.
> >
> > 1. I checked the code of KafkaProducer#doSend()
> > and RecordAccumulator#append(), if KafkaProducer#doSend() returns it
> means
> > serializedKey and serializedValue have been appended to
> > ProducerBatch#recordsBuilder and we don't keep reference of serializedKey
> > and serializedValue.
> >
> > 2. According to 1, the user application can reuse the ByteBuffer to send
> > consecutive KafkaProducer#send() requests without breaking the user
> > application. If we are concerned about compatibility, we can provide
> > another Serializer, such as ZeroCopyByteBufferSerializer, and keep the
> > original ByteBufferSerializer unchanged.
> >
> > In my opinion, kafka-clients should provide some way for users who want
> to
> > improve application performance, if users want to improve application
> > performance, they should use lower level code and understand the
> underlying
> > implementation of these codes.
> >
> > Best,
> > ShunKang
> >
> > Divij Vaidya  于2022年9月28日周三 19:58写道:
> >
> > > Hello
> > >
> > > I believe that the current behaviour of creating a copy of the user
> > > provided input is the correct approach because of the following
> reasons:
> > >
> > > 1. In the existing implementation (considering cases when T is
> ByteBuffer
> > > in Serializer#serialize(String,Headers,T)) we copy the data (T) into a
> > new
> > > byte[]. In the new approach, we would continue to re-use the ByteBuffer
> > > even after doSend() which means the `ProducerRecord` object cannot go
> out
> > > of scope from a GC perspective at the end of doSend(). Hence, the new
> > > approach may lead to increased heap memory usage for a greater period
> of
> > > time.
> > >
> > > 2. The new approach may break certain user applications e.g. consider
> an
> > > user application which re-uses the ByteBuffer (maybe it's a memory
> mapped
> > > byte buffer) to send consecutive Producer.send() requests. Prior to
> this
> > > change, they could do that because we copy the data from user provided
> > > input before storing it in the accumulator but after this change, they
> > will
> > > have to allocate a new ByteBuffer for every ProduceRecord.
> > >
> > > In general, I am of the opinion that any user provided data should be
> > > copied to internal data structures at the interface of an opaque
> library
> > > (like client) so that the user doesn't have to guess about memory
> > lifetime
> > > of the objects they provided to the opaque library.
> > >
> > > What do you think?
> > >
> > > --
> > > Divij Vaidya
> > >
> > >
> > >
> > > On Sun, Sep 25, 2022 at 5:59 PM 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
> > > >
> > >
> >
>


Re: [DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-09-28 Thread ShunKang Lin
Hi Divij Vaidya,

Thanks for your comments.

1. I checked the code of KafkaProducer#doSend()
and RecordAccumulator#append(), if KafkaProducer#doSend() returns it means
serializedKey and serializedValue have been appended to
ProducerBatch#recordsBuilder and we don't keep reference of serializedKey
and serializedValue.

2. According to 1, the user application can reuse the ByteBuffer to send
consecutive KafkaProducer#send() requests without breaking the user
application. If we are concerned about compatibility, we can provide
another Serializer, such as ZeroCopyByteBufferSerializer, and keep the
original ByteBufferSerializer unchanged.

In my opinion, kafka-clients should provide some way for users who want to
improve application performance, if users want to improve application
performance, they should use lower level code and understand the underlying
implementation of these codes.

Best,
ShunKang

Divij Vaidya  于2022年9月28日周三 19:58写道:

> Hello
>
> I believe that the current behaviour of creating a copy of the user
> provided input is the correct approach because of the following reasons:
>
> 1. In the existing implementation (considering cases when T is ByteBuffer
> in Serializer#serialize(String,Headers,T)) we copy the data (T) into a new
> byte[]. In the new approach, we would continue to re-use the ByteBuffer
> even after doSend() which means the `ProducerRecord` object cannot go out
> of scope from a GC perspective at the end of doSend(). Hence, the new
> approach may lead to increased heap memory usage for a greater period of
> time.
>
> 2. The new approach may break certain user applications e.g. consider an
> user application which re-uses the ByteBuffer (maybe it's a memory mapped
> byte buffer) to send consecutive Producer.send() requests. Prior to this
> change, they could do that because we copy the data from user provided
> input before storing it in the accumulator but after this change, they will
> have to allocate a new ByteBuffer for every ProduceRecord.
>
> In general, I am of the opinion that any user provided data should be
> copied to internal data structures at the interface of an opaque library
> (like client) so that the user doesn't have to guess about memory lifetime
> of the objects they provided to the opaque library.
>
> What do you think?
>
> --
> Divij Vaidya
>
>
>
> On Sun, Sep 25, 2022 at 5:59 PM ShunKang Lin 
> 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
> >
>


Re: [DISCUSS] KIP-871: Fix ByteBufferSerializer#serialize(String, ByteBuffer) compatible problem

2022-09-27 Thread ShunKang Lin
Hi Guozhang,

Thanks for your comments!

I saw this Jira has been open for 5 years so I thought maybe a KIP was
needed. I will discard this KIP now.

Best,
ShunKang

Guozhang Wang  于2022年9月27日周二 06:27写道:

> Hello ShunKang,
>
> Thanks for filing the report. I looked at the source code and I agree it's
> a bug. The reason we did not get it in the past is probably just because we
> were inefficiently keep byte-copying and hence for almost all cases, the
> offset is set to 0 when serialization happens.
>
> Note that this fix does not really require a KIP since it does not change
> any public APIs (for details on what kind of changes require a KIP
> discussion, please see
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-Whatisconsidereda%22majorchange%22thatneedsaKIP
> ?
> <https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals#KafkaImprovementProposals-Whatisconsidereda%22majorchange%22thatneedsaKIP?>
> )
>
>
> Guozhang
>
> On Sun, Sep 25, 2022 at 6:52 AM ShunKang Lin 
> wrote:
>
> > Hi all,
> >
> > I'd like to start a new discussion thread on KIP-871 (Kafka Client) which
> > proposes that fix ByteBufferSerializer#serialize(String, ByteBuffer)
> > compatible problem.
> >
> > Links:
> >
> > KIP:
> >
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495816
> >
> > Jira: https://issues.apache.org/jira/browse/KAFKA-4852
> >
> > Thanks,
> > ShunKang
> >
>
>
> --
> -- Guozhang
>


[DISCUSS] KIP-872: Add Serializer#serializeToByteBuffer() to reduce memory copying

2022-09-25 Thread ShunKang Lin
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


[DISCUSS] KIP-871: Fix ByteBufferSerializer#serialize(String, ByteBuffer) compatible problem

2022-09-25 Thread ShunKang Lin
Hi all,

I'd like to start a new discussion thread on KIP-871 (Kafka Client) which
proposes that fix ByteBufferSerializer#serialize(String, ByteBuffer)
compatible problem.

Links:

KIP:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=228495816

Jira: https://issues.apache.org/jira/browse/KAFKA-4852

Thanks,
ShunKang


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-25 Thread ShunKang Lin
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  于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  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 
> > 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
>


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-22 Thread ShunKang Lin
Thanks Guozhang!

Best,
ShunKang

Guozhang Wang  于2022年9月23日周五 00:27写道:

> Could you start a separate VOTE email thread calling for votes?
>
> On Thu, Sep 22, 2022 at 9:19 AM ShunKang Lin 
> wrote:
>
> > Hi Guozhang,
> >
> > Thanks for your help! By the way, what should I do next?
> >
> > Best,
> > ShunKang
> >
> > Guozhang Wang  于2022年9月22日周四 23:21写道:
> >
> > > Thanks ShunKang,
> > >
> > > I made a few nit edits on the Motivation section as well. LGTM for me
> > now.
> > >
> > > On Thu, Sep 22, 2022 at 7:33 AM ShunKang Lin <
> linshunkang@gmail.com>
> > > wrote:
> > >
> > > > Hi Guozhang,
> > > >
> > > > I've updated the "Motivation" section of the KIP, please take a look.
> > > >
> > > > Thanks.
> > > > ShunKang
> > > >
> > > > Guozhang Wang  于2022年9月21日周三 01:26写道:
> > > >
> > > > > In this case, could you update the KIP to clarify the allocation
> > > savings
> > > > > more clearly in the "Motivation" section? Also you could mention
> that
> > > for
> > > > > user customizable serdes, if they could provide overwrites on the
> > > > > overloaded function that's also possible for optimize memory
> > > allocations.
> > > > >
> > > > > Guozhang
> > > > >
> > > > > On Tue, Sep 20, 2022 at 10:24 AM Guozhang Wang  >
> > > > wrote:
> > > > >
> > > > > > 1. Ack, thanks.
> > > > > > 2. Sounds good, thanks for clarifying.
> > > > > >
> > > > > > On Tue, Sep 20, 2022 at 9:50 AM ShunKang Lin <
> > > > linshunkang@gmail.com>
> > > > > > wrote:
> > > > > >
> > > > > >> Hi Guozhang,
> > > > > >>
> > > > > >> Thanks for your comments!
> > > > > >>
> > > > > >> 1. We can reduce memory allocation if the key/value types happen
> > to
> > > be
> > > > > >> ByteBuffer or String.
> > > > > >> 2. I would like to add `default ByteBuffer
> > > > serializeToByteBuffer(String
> > > > > >> topic, Headers headers, T data)` in Serializer to reduce memory
> > copy
> > > > in
> > > > > >> `KafkaProducer#doSend(ProducerRecord, Callback)`, but this
> change
> > > is a
> > > > > bit
> > > > > >> big, I prefer to submit another one KIP to do the job.
> > > > > >>
> > > > > >> Thanks.
> > > > > >> ShunKang
> > > > > >>
> > > > > >> Guozhang Wang  于2022年9月20日周二 06:32写道:
> > > > > >>
> > > > > >> > 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
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > > -- Guozhang
> > > > >
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>


[VOTE] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-22 Thread ShunKang Lin
Hi everyone,

I'd like to open the vote for KIP-863, which proposes to reduce memory
allocation and memory copying in Fetcher#parseRecord(TopicPartition,
RecordBatch, Record).

The proposal is here:
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=225152035

Thanks to all who reviewed the proposal, and thanks in advance for taking
the time to vote!

Best,
ShunKang


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-22 Thread ShunKang Lin
Hi Guozhang,

Thanks for your help! By the way, what should I do next?

Best,
ShunKang

Guozhang Wang  于2022年9月22日周四 23:21写道:

> Thanks ShunKang,
>
> I made a few nit edits on the Motivation section as well. LGTM for me now.
>
> On Thu, Sep 22, 2022 at 7:33 AM ShunKang Lin 
> wrote:
>
> > Hi Guozhang,
> >
> > I've updated the "Motivation" section of the KIP, please take a look.
> >
> > Thanks.
> > ShunKang
> >
> > Guozhang Wang  于2022年9月21日周三 01:26写道:
> >
> > > In this case, could you update the KIP to clarify the allocation
> savings
> > > more clearly in the "Motivation" section? Also you could mention that
> for
> > > user customizable serdes, if they could provide overwrites on the
> > > overloaded function that's also possible for optimize memory
> allocations.
> > >
> > > Guozhang
> > >
> > > On Tue, Sep 20, 2022 at 10:24 AM Guozhang Wang 
> > wrote:
> > >
> > > > 1. Ack, thanks.
> > > > 2. Sounds good, thanks for clarifying.
> > > >
> > > > On Tue, Sep 20, 2022 at 9:50 AM ShunKang Lin <
> > linshunkang@gmail.com>
> > > > wrote:
> > > >
> > > >> Hi Guozhang,
> > > >>
> > > >> Thanks for your comments!
> > > >>
> > > >> 1. We can reduce memory allocation if the key/value types happen to
> be
> > > >> ByteBuffer or String.
> > > >> 2. I would like to add `default ByteBuffer
> > serializeToByteBuffer(String
> > > >> topic, Headers headers, T data)` in Serializer to reduce memory copy
> > in
> > > >> `KafkaProducer#doSend(ProducerRecord, Callback)`, but this change
> is a
> > > bit
> > > >> big, I prefer to submit another one KIP to do the job.
> > > >>
> > > >> Thanks.
> > > >> ShunKang
> > > >>
> > > >> Guozhang Wang  于2022年9月20日周二 06:32写道:
> > > >>
> > > >> > 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
> > > >
> > >
> > >
> > > --
> > > -- Guozhang
> > >
> >
>
>
> --
> -- Guozhang
>


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-22 Thread ShunKang Lin
Hi Guozhang,

I've updated the "Motivation" section of the KIP, please take a look.

Thanks.
ShunKang

Guozhang Wang  于2022年9月21日周三 01:26写道:

> In this case, could you update the KIP to clarify the allocation savings
> more clearly in the "Motivation" section? Also you could mention that for
> user customizable serdes, if they could provide overwrites on the
> overloaded function that's also possible for optimize memory allocations.
>
> Guozhang
>
> On Tue, Sep 20, 2022 at 10:24 AM Guozhang Wang  wrote:
>
> > 1. Ack, thanks.
> > 2. Sounds good, thanks for clarifying.
> >
> > On Tue, Sep 20, 2022 at 9:50 AM ShunKang Lin 
> > wrote:
> >
> >> Hi Guozhang,
> >>
> >> Thanks for your comments!
> >>
> >> 1. We can reduce memory allocation if the key/value types happen to be
> >> ByteBuffer or String.
> >> 2. I would like to add `default ByteBuffer serializeToByteBuffer(String
> >> topic, Headers headers, T data)` in Serializer to reduce memory copy in
> >> `KafkaProducer#doSend(ProducerRecord, Callback)`, but this change is a
> bit
> >> big, I prefer to submit another one KIP to do the job.
> >>
> >> Thanks.
> >> ShunKang
> >>
> >> Guozhang Wang  于2022年9月20日周二 06:32写道:
> >>
> >> > 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
> >
>
>
> --
> -- Guozhang
>


Re: [DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-09-20 Thread ShunKang Lin
Hi Guozhang,

Thanks for your comments!

1. We can reduce memory allocation if the key/value types happen to be
ByteBuffer or String.
2. I would like to add `default ByteBuffer serializeToByteBuffer(String
topic, Headers headers, T data)` in Serializer to reduce memory copy in
`KafkaProducer#doSend(ProducerRecord, Callback)`, but this change is a bit
big, I prefer to submit another one KIP to do the job.

Thanks.
ShunKang

Guozhang Wang  于2022年9月20日周二 06:32写道:

> 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 
> 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
>


[DISCUSS] KIP-863: Reduce Fetcher#parseRecord() memory copy

2022-08-21 Thread ShunKang Lin
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.


Re: Granting permission for Create KIP

2022-08-21 Thread ShunKang Lin
Thanks.

Chris Egerton  于2022年8月21日周日 23:00写道:

> I've added you to the Wiki and Jira; you should be good to go now.
>
> On Sun, Aug 21, 2022 at 10:02 AM ShunKang Lin 
> wrote:
>
> > Hi all,
> > Please grant permission for Create KIP to wiki ID: LinShunKang.
> >
> > Thanks.
> >
>


Granting permission for Create KIP

2022-08-21 Thread ShunKang Lin
Hi all,
Please grant permission for Create KIP to wiki ID: LinShunKang.

Thanks.