So far the discussion was around the performance characteristics of the new
compression algorithm. Another area that is important and is not covered in
the KIP is the compatibility implications. For example, what happens if a
consumer that doesn't support zstd tries to consume a topic compressed with
it? Or if a broker that doesn't support receives data compressed with it?
If we go through that exercise, then more changes may be required (like
bumping the version of produce/fetch protocols).

Ismael

On Wed, Jan 25, 2017 at 3:22 PM, Ben Stopford <b...@confluent.io> wrote:

> Is there more discussion to be had on this KIP, or should it be taken to a
> vote?
>
> On Mon, Jan 16, 2017 at 6:37 AM Dongjin Lee <dong...@apache.org> wrote:
>
> > I updated KIP-110 with JMH-measured benchmark results. Please have a
> review
> > when you are free. (The overall result is not different yet.)
> >
> > Regards,
> > Dongjin
> >
> > +1. Could anyone assign KAFKA-4514 to me?
> >
> > On Thu, Jan 12, 2017 at 11:39 AM, Dongjin Lee <dong...@apache.org>
> wrote:
> >
> > > Okay, I will have a try.
> > > Thanks Ewen for the guidance!!
> > >
> > > Best,
> > > Dongjin
> > >
> > > On Thu, Jan 12, 2017 at 6:44 AM, Ismael Juma <ism...@juma.me.uk>
> wrote:
> > >
> > >> That's a good point Ewen. Dongjin, you could use the branch that Ewen
> > >> linked for the performance testing. It would also help validate the
> PR.
> > >>
> > >> Ismael
> > >>
> > >> On Wed, Jan 11, 2017 at 9:38 PM, Ewen Cheslack-Postava <
> > e...@confluent.io
> > >> >
> > >> wrote:
> > >>
> > >> > FYI, there's an outstanding patch for getting some JMH benchmarking
> > >> setup:
> > >> > https://github.com/apache/kafka/pull/1712 I haven't found time to
> > >> review
> > >> > it
> > >> > (and don't really know JMH well anyway) but it might be worth
> getting
> > >> that
> > >> > landed so we can use it for this as well.
> > >> >
> > >> > -Ewen
> > >> >
> > >> > On Wed, Jan 11, 2017 at 6:35 AM, Dongjin Lee <dong...@apache.org>
> > >> wrote:
> > >> >
> > >> > > Hi Ismael,
> > >> > >
> > >> > > 1. In the case of compression output, yes, lz4 is producing the
> > >> smaller
> > >> > > output than gzip. In fact, my benchmark was inspired
> > >> > > by MessageCompressionTest#testCompressSize unit test and the
> result
> > >> is
> > >> > > same - 396 bytes for gzip and 387 bytes for lz4.
> > >> > > 2. I agree that my (former) approach can result in unreliable
> > output.
> > >> > > However, I am experiencing difficulties on how to acquire the
> > >> benchmark
> > >> > > metrics from Kafka. For you recommended JMH, I just started to
> > google
> > >> for
> > >> > > it. If possible, could you give any example on how to use JMH
> > against
> > >> > > Kafka? If it is the case, it will be a great help.
> > >> > > Regards,Dongjin
> > >> > >
> > >> > >                 _____________________________
> > >> > > From: Ismael Juma <ism...@juma.me.uk>
> > >> > > Sent: Wednesday, January 11, 2017 7:33 PM
> > >> > > Subject: Re: [DISCUSS] KIP-110: Add Codec for ZStandard
> Compression
> > >> > > To:  <dev@kafka.apache.org>
> > >> > >
> > >> > >
> > >> > > Thanks Dongjin. I highly recommend using JMH for the benchmark,
> the
> > >> > > existing one has a few problems that could result in unreliable
> > >> results.
> > >> > > Also, it's a bit surprising that LZ4 is producing smaller output
> > than
> > >> > gzip.
> > >> > > Is that right?
> > >> > >
> > >> > > Ismael
> > >> > >
> > >> > > On Wed, Jan 11, 2017 at 10:20 AM, Dongjin Lee <dong...@apache.org
> >
> > >> > wrote:
> > >> > >
> > >> > > > Ismael,
> > >> > > >
> > >> > > > I pushed the benchmark code I used, with some updates
> (iteration:
> > >> 20 ->
> > >> > > > 1000). I also updated the KIP page with the updated benchmark
> > >> results.
> > >> > > > Please take a review when you are free. The attached screenshot
> > >> shows
> > >> > how
> > >> > > > to run the benchmarker.
> > >> > > >
> > >> > > > Thanks,
> > >> > > > Dongjin
> > >> > > >
> > >> > > > On Tue, Jan 10, 2017 at 8:03 PM, Dongjin Lee <
> dong...@apache.org>
> > >> > wrote:
> > >> > > >
> > >> > > >> Ismael,
> > >> > > >>
> > >> > > >> I see. Then, I will share the benchmark code I used by
> tomorrow.
> > >> > Thanks
> > >> > > >> for your guidance.
> > >> > > >>
> > >> > > >> Best,
> > >> > > >> Dongjin
> > >> > > >>
> > >> > > >> -----
> > >> > > >>
> > >> > > >> Dongjin Lee
> > >> > > >>
> > >> > > >> Software developer in Line+.
> > >> > > >> So interested in massive-scale machine learning.
> > >> > > >>
> > >> > > >> facebook: www.facebook.com/dongjin.lee.kr
> > >> > > >> linkedin: kr.linkedin.com/in/dongjinleekr
> > >> > > >> github: github.com/dongjinleekr
> > >> > > >> twitter: www.twitter.com/dongjinleekr
> > >> > > >>
> > >> > > >>
> > >> > > >>
> > >> > > >>
> > >> > > >> On Tue, Jan 10, 2017 at 7:24 PM +0900, "Ismael Juma" <
> > >> > ism...@juma.me.uk
> > >> > > >
> > >> > > >> wrote:
> > >> > > >>
> > >> > > >> Dongjin,
> > >> > > >>>
> > >> > > >>> The KIP states:
> > >> > > >>>
> > >> > > >>> "I compared the compressed size and compression time of 3
> > >> 1kb-sized
> > >> > > >>> messages (3102 bytes in total), with the Draft-implementation
> of
> > >> > > ZStandard
> > >> > > >>> Compression Codec and all currently available
> CompressionCodecs.
> > >> All
> > >> > > >>> elapsed times are the average of 20 trials."
> > >> > > >>>
> > >> > > >>> But doesn't give any details of how this was implemented. Is
> the
> > >> > source
> > >> > > >>> code available somewhere? Micro-benchmarking in the JVM is
> > pretty
> > >> > > tricky so
> > >> > > >>> it needs verification before numbers can be trusted. A
> > performance
> > >> > test
> > >> > > >>> with kafka-producer-perf-test.sh would be nice to have as
> well,
> > if
> > >> > > possible.
> > >> > > >>>
> > >> > > >>> Thanks,
> > >> > > >>> Ismael
> > >> > > >>>
> > >> > > >>> On Tue, Jan 10, 2017 at 7:44 AM, Dongjin Lee  wrote:
> > >> > > >>>
> > >> > > >>> > Ismael,
> > >> > > >>> >
> > >> > > >>> > 1. Is the benchmark in the KIP page not enough? You mean we
> > >> need a
> > >> > > whole
> > >> > > >>> > performance test using kafka-producer-perf-test.sh?
> > >> > > >>> >
> > >> > > >>> > 2. It seems like no major project is relying on it
> currently.
> > >> > > However,
> > >> > > >>> > after reviewing the code, I concluded that at least this
> > project
> > >> > has
> > >> > > a good
> > >> > > >>> > test coverage. And for the problem of upstream tracking -
> > >> although
> > >> > > there is
> > >> > > >>> > no significant update on ZStandard to judge this problem, it
> > >> seems
> > >> > > not bad.
> > >> > > >>> > If required, I can take responsibility of the tracking for
> > this
> > >> > > library.
> > >> > > >>> >
> > >> > > >>> > Thanks,
> > >> > > >>> > Dongjin
> > >> > > >>> >
> > >> > > >>> > On Tue, Jan 10, 2017 at 7:09 AM, Ismael Juma  wrote:
> > >> > > >>> >
> > >> > > >>> > > Thanks for posting the KIP, ZStandard looks like a nice
> > >> > > improvement over
> > >> > > >>> > > the existing compression algorithms. A couple of
> questions:
> > >> > > >>> > >
> > >> > > >>> > > 1. Can you please elaborate on the details of the
> benchmark?
> > >> > > >>> > > 2. About https://github.com/luben/zstd-jni, can we rely
> on
> > >> it? A
> > >> > > few
> > >> > > >>> > > things
> > >> > > >>> > > to consider: are there other projects using it, does it
> have
> > >> good
> > >> > > test
> > >> > > >>> > > coverage, are there performance tests, does it track
> > upstream
> > >> > > closely?
> > >> > > >>> > >
> > >> > > >>> > > Thanks,
> > >> > > >>> > > Ismael
> > >> > > >>> > >
> > >> > > >>> > > On Fri, Jan 6, 2017 at 2:40 AM, Dongjin Lee  wrote:
> > >> > > >>> > >
> > >> > > >>> > > > Hi all,
> > >> > > >>> > > >
> > >> > > >>> > > > I've just posted a new KIP "KIP-110: Add Codec for
> > ZStandard
> > >> > > >>> > Compression"
> > >> > > >>> > > > for
> > >> > > >>> > > > discussion:
> > >> > > >>> > > >
> > >> > > >>> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >> > > >>> > > > 110%3A+Add+Codec+for+ZStandard+Compression
> > >> > > >>> > > >
> > >> > > >>> > > > Please have a look when you are free.
> > >> > > >>> > > >
> > >> > > >>> > > > Best,
> > >> > > >>> > > > Dongjin
> > >> > > >>> > > >
> > >> > > >>> > > > --
> > >> > > >>> > > > *Dongjin Lee*
> > >> > > >>> > > >
> > >> > > >>> > > >
> > >> > > >>> > > > *Software developer in Line+.So interested in
> > massive-scale
> > >> > > machine
> > >> > > >>> > > > learning.facebook: www.facebook.com/dongjin.lee.kr
> > >> > > >>> > > > linkedin:
> > >> > > >>> > > > kr.linkedin.com/in/dongjinleekr
> > >> > > >>> > > > github:
> > >> > > >>> > > > github.com/dongjinleekr
> > >> > > >>> > > > twitter: www.twitter.com/dongjinleekr
> > >> > > >>> > > > *
> > >> > > >>> > > >
> > >> > > >>> > >
> > >> > > >>> >
> > >> > > >>> >
> > >> > > >>> >
> > >> > > >>> > --
> > >> > > >>> > *Dongjin Lee*
> > >> > > >>> >
> > >> > > >>> >
> > >> > > >>> > *Software developer in Line+.So interested in massive-scale
> > >> machine
> > >> > > >>> > learning.facebook: www.facebook.com/dongjin.lee.kr
> > >> > > >>> > linkedin:
> > >> > > >>> > kr.linkedin.com/in/dongjinleekr
> > >> > > >>> > github:
> > >> > > >>> > github.com/dongjinleekr
> > >> > > >>> > twitter: www.twitter.com/dongjinleekr
> > >> > > >>> > *
> > >> > > >>> >
> > >> > > >>>
> > >> > > >>>
> > >> > > >
> > >> > > >
> > >> > > > --
> > >> > > > *Dongjin Lee*
> > >> > > >
> > >> > > >
> > >> > > > *Software developer in Line+.So interested in massive-scale
> > machine
> > >> > > > learning.facebook: www.facebook.com/dongjin.lee.kr
> > >> > > > <http://www.facebook.com/dongjin.lee.kr>linkedin:
> > >> kr.linkedin.com/in/
> > >> > > dongjinleekr
> > >> > > > <http://kr.linkedin.com/in/dongjinleekr>github:
> > >> > > > <http://goog_969573159/>github.com/dongjinleekr
> > >> > > > <http://github.com/dongjinleekr>twitter:
> > >> www.twitter.com/dongjinleekr
> > >> > > > <http://www.twitter.com/dongjinleekr>*
> > >> > > >
> > >> > >
> > >> > >
> > >> > >
> > >> > >
> > >> > >
> > >> >
> > >>
> > >
> > >
> > >
> > > --
> > > *Dongjin Lee*
> > >
> > >
> > > *Software developer in Line+.So interested in massive-scale machine
> > > learning.facebook: www.facebook.com/dongjin.lee.kr
> > > <http://www.facebook.com/dongjin.lee.kr>linkedin:
> > kr.linkedin.com/in/dongjinleekr
> > > <http://kr.linkedin.com/in/dongjinleekr>github:
> > > <http://goog_969573159/>github.com/dongjinleekr
> > > <http://github.com/dongjinleekr>twitter: www.twitter.com/dongjinleekr
> > > <http://www.twitter.com/dongjinleekr>*
> > >
> >
> >
> >
> > --
> > *Dongjin Lee*
> >
> >
> > *Software developer in Line+.So interested in massive-scale machine
> > learning.facebook: www.facebook.com/dongjin.lee.kr
> > <http://www.facebook.com/dongjin.lee.kr>linkedin:
> > kr.linkedin.com/in/dongjinleekr
> > <http://kr.linkedin.com/in/dongjinleekr>github:
> > <http://goog_969573159/>github.com/dongjinleekr
> > <http://github.com/dongjinleekr>twitter: www.twitter.com/dongjinleekr
> > <http://www.twitter.com/dongjinleekr>*
> >
>

Reply via email to