I see. Let me have a check. If not needed, of course, we don't have to
waste on configuration options.

Since the KIP deadline is imminent, I just opened the voting thread. Let's
continue the discussion here.

Best,
Dongjin

On Mon, Jan 21, 2019 at 1:30 AM Ismael Juma <ism...@juma.me.uk> wrote:

> Hi Dongjin,
>
> When the compression type is "producer", then the broker doesn't recompress
> though. Thinking about it some more, there are some uncommon cases where
> recompression does happen (the old (and hopefully hardly used by now)
> message format == 0 and some edge cases), so it is a good point you raised.
>
> It's a bit unfortunate to add so many topic configs for cases that probably
> don't matter. That is, if you are using "producer" compression, you
> probably don't need to configure these settings and can live with the
> defaults. Perhaps we should only support the topic config for the cases
> where you are actually recompressing in the broker.
>
> What do you think? I'd be interested in other people's thoughts too.
>
> Ismael
>
> On Sun, Jan 20, 2019 at 2:14 AM Dongjin Lee <dong...@apache.org> wrote:
>
> > Hi Ismael,
> >
> > It seems like it needs more explanation. Here is the detailed reasoning.
> >
> > You know, topic and broker's 'compression.type' allows 'uncompressed',
> > 'producer' with standard codecs (i.e., gzip, snappy, lz4, zstd.) And this
> > configuration is used by the broker in the re-compressing process after
> > offset assignment. After this feature, the new configs,
> 'compression.level'
> > and 'compression.buffer.size', also will be used in this process.
> >
> > The problem arises when given topic's compression type (whether it was
> > inherited from broker's configuration or explicitly set) is 'producer.'
> > With this setting, the compression codec to be used is decided by the
> > producer client. Since there is no way to restore the compression level
> and
> > buffer size from the message, we can take the following strategies:
> >
> > 1. Just use given 'compression.level' and 'compression.buffer.size'
> > settings.
> >
> > It will cause so many errors. Let's imagine the case of topic's
> > configuration is { compression.type=producer, compression.level=10,
> > compression.buffer.size=8192 }. In this case, all producers with gzip or
> > lz4 compressed messages will result in an error. (gzip doesn't allow
> > compression level 10, and lz4 also for a buffer size of 8192.)
> >
> > 2. Extend the message format to include compression configurations.
> >
> > With this strategy, we need to change the message format - it's a too big
> > change.
> >
> > 3. If topic's compression.type is 'producer', use the default
> configuration
> > for the given codec.
> >
> > With this strategy, allowing fine-grained compression configuration is
> > meaningless.
> >
> > For the above reasons, I think the only alternative is providing options
> > that can be used when the topic's 'compression.type' is 'producer.' In
> > other words, adding compression.[gzip, lz4, zstd].level and
> > compression.[gzip.snappy.lz4].buffer.size options - and it is what I did
> in
> > the last modification.
> >
> > (wait, the reasoning above should be included in the KIP in the rejected
> > alternatives section, isn't it?)
> >
> > Thanks,
> > Dongjin
> >
> > On Sun, Jan 20, 2019 at 2:33 AM Ismael Juma <ism...@juma.me.uk> wrote:
> >
> > > Hi Dongjin,
> > >
> > > For topic level, you can only have a single compression type so the way
> > it
> > > was before was fine, right? The point you raise is how to set broker
> > > defaults that vary depending on the compression type, correct?
> > >
> > > Ismael
> > >
> > > On Mon, Jan 14, 2019 at 10:18 AM Dongjin Lee <dong...@apache.org>
> wrote:
> > >
> > > > I just realized that there was a missing hole in the KIP, so I fixed
> > it.
> > > > The draft implementation will be updated soon.
> > > >
> > > > In short, the proposed change did not regard the case of the topic or
> > > > broker's 'compression.type' is 'producer'; in this case, the broker
> has
> > > to
> > > > handle all kinds of the supported codec. So I added additional
> options
> > > > (compression.[gzip,snappy,lz4, zstd].level,
> > compression.[gzip,snappy,lz4,
> > > > zstd].buffer.size) with handling routines.
> > > >
> > > > Please have a look when you are free.
> > > >
> > > > Thanks,
> > > > Dongjin
> > > >
> > > > On Mon, Jan 7, 2019 at 6:23 AM Dongjin Lee <dong...@apache.org>
> wrote:
> > > >
> > > > > Thanks for pointing out Ismael. It's now updated.
> > > > >
> > > > > Best,
> > > > > Dongjin
> > > > >
> > > > > On Mon, Jan 7, 2019 at 4:36 AM Ismael Juma <isma...@gmail.com>
> > wrote:
> > > > >
> > > > >> Thanks Dongjin. One minor suggestion: we should mention that the
> > > broker
> > > > >> side configs are also topic configs (i.e. can be set for a given
> > > topic).
> > > > >>
> > > > >> Ismael
> > > > >>
> > > > >> On Sun, Jan 6, 2019, 10:37 AM Dongjin Lee <dong...@apache.org
> > wrote:
> > > > >>
> > > > >> > Happy new year.
> > > > >> >
> > > > >> > I just updated the title and contents of KIP and Jira issue,
> with
> > > > >> updated
> > > > >> > draft implementation. Now both of compression level and buffer
> > size
> > > > >> options
> > > > >> > are available to producer and broker configuration. You can
> check
> > > the
> > > > >> > updated KIP from modified URL:
> > > > >> >
> > > > >> >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-390%3A+Allow+fine-grained+configuration+for+compression
> > > > >> >
> > > > >> > Please have a look when you are free.
> > > > >> >
> > > > >> > Thanks,
> > > > >> > Dongjin
> > > > >> >
> > > > >> > On Mon, Dec 3, 2018 at 12:50 AM Ismael Juma <isma...@gmail.com>
> > > > wrote:
> > > > >> >
> > > > >> > > The updated title sounds fine to me.
> > > > >> > >
> > > > >> > > Ismael
> > > > >> > >
> > > > >> > > On Sun, Dec 2, 2018, 5:25 AM Dongjin Lee <dong...@apache.org
> > > wrote:
> > > > >> > >
> > > > >> > > > Hi Ismael,
> > > > >> > > >
> > > > >> > > > Got it. Your direction is perfectly reasonable. I am now
> > > updating
> > > > >> the
> > > > >> > KIP
> > > > >> > > > document and the implementation.
> > > > >> > > >
> > > > >> > > > By allowing the buffer/block size to be configurable, it
> would
> > > be
> > > > >> > better
> > > > >> > > to
> > > > >> > > > update the title of the KIP like 'Allow fine-grained
> > > configuration
> > > > >> for
> > > > >> > > > compression'. Is that right?
> > > > >> > > >
> > > > >> > > > @Other committers:
> > > > >> > > >
> > > > >> > > > Is there any other opinion on allowing the buffer/block size
> > to
> > > be
> > > > >> > > > configurable?
> > > > >> > > >
> > > > >> > > > Thanks,
> > > > >> > > > Dongjin
> > > > >> > > >
> > > > >> > > > On Thu, Nov 29, 2018 at 1:45 AM Ismael Juma <
> > ism...@juma.me.uk>
> > > > >> wrote:
> > > > >> > > >
> > > > >> > > > > Hi Dongjin,
> > > > >> > > > >
> > > > >> > > > > To clarify, I mean a broker topic config with regards to
> > point
> > > > 1.
> > > > >> As
> > > > >> > > you
> > > > >> > > > > know, compression can be done by the producer and/or by
> the
> > > > >> broker.
> > > > >> > The
> > > > >> > > > > default is for the broker to just use whatever compression
> > was
> > > > >> used
> > > > >> > by
> > > > >> > > > the
> > > > >> > > > > producer, but this can be changed by the user on a per
> topic
> > > > >> basis.
> > > > >> > It
> > > > >> > > > > seems like it would make sense for the configs to be .
> > > > consistent
> > > > >> > > between
> > > > >> > > > > producer and broker.
> > > > >> > > > >
> > > > >> > > > > For point 2, I haven't looked at the implementation, but
> we
> > > > could
> > > > >> do
> > > > >> > it
> > > > >> > > > in
> > > > >> > > > > the `CompressionType` enum by invoking the right
> constructor
> > > or
> > > > >> > > > retrieving
> > > > >> > > > > the default value via a constant (if defined). That's an
> > > > >> > implementation
> > > > >> > > > > detail and can be discussed in the PR. The more general
> > point
> > > is
> > > > >> to
> > > > >> > > rely
> > > > >> > > > on
> > > > >> > > > > the library defaults instead of choosing one ourselves.
> > > > >> > > > >
> > > > >> > > > > For point 3, I'm in favour of doing that in this KIP.
> > > > >> > > > >
> > > > >> > > > > Ismael
> > > > >> > > > >
> > > > >> > > > > On Wed, Nov 28, 2018 at 7:01 AM Dongjin Lee <
> > > dong...@apache.org
> > > > >
> > > > >> > > wrote:
> > > > >> > > > >
> > > > >> > > > > > Thank you Ismael, here are the answers:
> > > > >> > > > > >
> > > > >> > > > > > *1. About topic config*
> > > > >> > > > > >
> > > > >> > > > > > After some consideration, I concluded that topic config
> > > > doesn't
> > > > >> > need
> > > > >> > > to
> > > > >> > > > > > support compression.level. Here is why: since the
> > > compression
> > > > is
> > > > >> > > > > conducted
> > > > >> > > > > > by the client, the one who can select the best
> compression
> > > > >> level is
> > > > >> > > the
> > > > >> > > > > > client itself. Let us assume that the compression level
> is
> > > set
> > > > >> at
> > > > >> > the
> > > > >> > > > > topic
> > > > >> > > > > > config level. In that case, there is a possibility that
> > the
> > > > >> > > compression
> > > > >> > > > > > level is not optimal for some producers. Actually,
> Kafka's
> > > go
> > > > >> > client
> > > > >> > > > also
> > > > >> > > > > > supports compression level functionality for the
> producer
> > > > config
> > > > >> > > only.
> > > > >> > > > > > <
> https://github.com/Shopify/sarama/blob/master/config.go>
> > > > >> (wait,
> > > > >> > do
> > > > >> > > we
> > > > >> > > > > > need
> > > > >> > > > > > to add this reasoning in the KIP, rejected alternatives
> > > > >> section?)
> > > > >> > > > > >
> > > > >> > > > > > *2. About default level*
> > > > >> > > > > >
> > > > >> > > > > > As of current draft implementation, the default
> > compression
> > > is
> > > > >> set
> > > > >> > on
> > > > >> > > > the
> > > > >> > > > > > CompressionType enum. Of course, changing this strategy
> > into
> > > > >> > relying
> > > > >> > > > on a
> > > > >> > > > > > method from the library to pick the default compression
> > > level
> > > > >> seems
> > > > >> > > > > > possible, like `GZIPBlockOutputStream` does. In this
> case,
> > > we
> > > > >> need
> > > > >> > to
> > > > >> > > > add
> > > > >> > > > > > similar wrapper class for zstd and modify lz4 the
> wrapper
> > > > also.
> > > > >> Add
> > > > >> > > to
> > > > >> > > > > > this, it seems like we need to explicitly state that we
> > > follow
> > > > >> the
> > > > >> > > > > default
> > > > >> > > > > > compression level of the codec in the documentation. Is
> > this
> > > > >> what
> > > > >> > you
> > > > >> > > > > > intended?
> > > > >> > > > > >
> > > > >> > > > > > *3. Whether to allow the buffer/block size to be
> > > configurable*
> > > > >> > > > > >
> > > > >> > > > > > Well, As of current draft implementation, the lz4 level
> is
> > > > >> > > implemented
> > > > >> > > > as
> > > > >> > > > > > block size; this is caused by my misunderstanding on
> lz4.
> > > > After
> > > > >> > > > reviewing
> > > > >> > > > > > lz4 today, I found that it also supports compression
> level
> > > of
> > > > >> 1~16
> > > > >> > > > > > (default: 1), not block size. I will fix it in this
> > weekend
> > > by
> > > > >> > > updating
> > > > >> > > > > the
> > > > >> > > > > > wrapper class.
> > > > >> > > > > >
> > > > >> > > > > > For the problem of the buffer/block size, I have no
> strong
> > > > >> opinion.
> > > > >> > > If
> > > > >> > > > > the
> > > > >> > > > > > community needs it, I will do it all together. How do
> you
> > > > think?
> > > > >> > > > > >
> > > > >> > > > > > In short, it seems like I need to update the KIP
> document
> > > for
> > > > >> issue
> > > > >> > > #1
> > > > >> > > > > and
> > > > >> > > > > > update the compression wrapper for issue #2, #3. Is this
> > > okay?
> > > > >> > > > > >
> > > > >> > > > > > Thanks,
> > > > >> > > > > > Dongjin
> > > > >> > > > > >
> > > > >> > > > > > On Wed, Nov 28, 2018 at 12:34 AM Ismael Juma <
> > > > isma...@gmail.com
> > > > >> >
> > > > >> > > > wrote:
> > > > >> > > > > >
> > > > >> > > > > > >  Thanks for the KIP, this is helpful. A few questions:
> > > > >> > > > > > >
> > > > >> > > > > > > 1. Have we considered whether we want to allow a
> similar
> > > > topic
> > > > >> > > > config?
> > > > >> > > > > > > 2. Can we rely on a method from the library to pick
> the
> > > > >> default
> > > > >> > > > > > compression
> > > > >> > > > > > > level if compression.level is not set? We do it for
> gzip
> > > and
> > > > >> it
> > > > >> > > would
> > > > >> > > > > > seem
> > > > >> > > > > > > reasonable to do something similar for the other
> > > compression
> > > > >> > > > libraries.
> > > > >> > > > > > > 3. Do we want to allow the buffer/block size to be
> > > > >> configurable?
> > > > >> > > This
> > > > >> > > > > has
> > > > >> > > > > > > an impact on memory usage and people may want to trade
> > > > >> > compression
> > > > >> > > > for
> > > > >> > > > > > > less/more memory in some cases. For example, the
> default
> > > for
> > > > >> LZ4
> > > > >> > is
> > > > >> > > > > 64KB
> > > > >> > > > > > > which is a bit high.
> > > > >> > > > > > >
> > > > >> > > > > > > Ismael
> > > > >> > > > > > >
> > > > >> > > > > > > On Sun, Nov 18, 2018, 2:07 PM Dongjin Lee <
> > > > dong...@apache.org
> > > > >> > > wrote:
> > > > >> > > > > > >
> > > > >> > > > > > > > Hello dev,
> > > > >> > > > > > > >
> > > > >> > > > > > > > I hope to initiate the discussion of KIP-390: Add
> > > producer
> > > > >> > option
> > > > >> > > > to
> > > > >> > > > > > > adjust
> > > > >> > > > > > > > compression level
> > > > >> > > > > > > > <
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-390%3A+Add+producer+option+to+adjust+compression+level
> > > > >> > > > > > > > >.
> > > > >> > > > > > > > All feedbacks will be highly appreciated.
> > > > >> > > > > > > >
> > > > >> > > > > > > > Best,
> > > > >> > > > > > > > Dongjin
> > > > >> > > > > > > >
> > > > >> > > > > > > > --
> > > > >> > > > > > > > *Dongjin Lee*
> > > > >> > > > > > > >
> > > > >> > > > > > > > *A hitchhiker in the mathematical world.*
> > > > >> > > > > > > >
> > > > >> > > > > > > > *github:  <http://goog_969573159/>
> > > github.com/dongjinleekr
> > > > >> > > > > > > > <http://github.com/dongjinleekr>linkedin:
> > > > >> > > > > > > kr.linkedin.com/in/dongjinleekr
> > > > >> > > > > > > > <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > >> > > > > > > > www.slideshare.net/dongjinleekr
> > > > >> > > > > > > > <http://www.slideshare.net/dongjinleekr>*
> > > > >> > > > > > > >
> > > > >> > > > > > >
> > > > >> > > > > >
> > > > >> > > > > >
> > > > >> > > > > > --
> > > > >> > > > > > *Dongjin Lee*
> > > > >> > > > > >
> > > > >> > > > > > *A hitchhiker in the mathematical world.*
> > > > >> > > > > > *github:  <http://goog_969573159/>
> github.com/dongjinleekr
> > > > >> > > > > > <https://github.com/dongjinleekr>linkedin:
> > > > >> > > > > kr.linkedin.com/in/dongjinleekr
> > > > >> > > > > > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > >> > > > > > speakerdeck.com/dongjin
> > > > >> > > > > > <https://speakerdeck.com/dongjin>*
> > > > >> > > > > >
> > > > >> > > > >
> > > > >> > > >
> > > > >> > > >
> > > > >> > > > --
> > > > >> > > > *Dongjin Lee*
> > > > >> > > >
> > > > >> > > > *A hitchhiker in the mathematical world.*
> > > > >> > > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > >> > > > <https://github.com/dongjinleekr>linkedin:
> > > > >> > > kr.linkedin.com/in/dongjinleekr
> > > > >> > > > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > >> > > > speakerdeck.com/dongjin
> > > > >> > > > <https://speakerdeck.com/dongjin>*
> > > > >> > > >
> > > > >> > >
> > > > >> >
> > > > >> >
> > > > >> > --
> > > > >> > *Dongjin Lee*
> > > > >> >
> > > > >> > *A hitchhiker in the mathematical world.*
> > > > >> > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > >> > <https://github.com/dongjinleekr>linkedin:
> > > > >> kr.linkedin.com/in/dongjinleekr
> > > > >> > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > >> > speakerdeck.com/dongjin
> > > > >> > <https://speakerdeck.com/dongjin>*
> > > > >> >
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > > *Dongjin Lee*
> > > > >
> > > > > *A hitchhiker in the mathematical world.*
> > > > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > <https://github.com/dongjinleekr>linkedin:
> > > > kr.linkedin.com/in/dongjinleekr
> > > > > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > speakerdeck.com/dongjin
> > > > > <https://speakerdeck.com/dongjin>*
> > > > >
> > > >
> > > >
> > > > --
> > > > *Dongjin Lee*
> > > >
> > > > *A hitchhiker in the mathematical world.*
> > > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > <https://github.com/dongjinleekr>linkedin:
> > > kr.linkedin.com/in/dongjinleekr
> > > > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > > > speakerdeck.com/dongjin
> > > > <https://speakerdeck.com/dongjin>*
> > > >
> > >
> >
> >
> > --
> > *Dongjin Lee*
> >
> > *A hitchhiker in the mathematical world.*
> > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > <https://github.com/dongjinleekr>linkedin:
> kr.linkedin.com/in/dongjinleekr
> > <https://kr.linkedin.com/in/dongjinleekr>speakerdeck:
> > speakerdeck.com/dongjin
> > <https://speakerdeck.com/dongjin>*
> >
>


-- 
*Dongjin Lee*

*A hitchhiker in the mathematical world.*
*github:  <http://goog_969573159/>github.com/dongjinleekr
<https://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/dongjinleekr
<https://kr.linkedin.com/in/dongjinleekr>speakerdeck: speakerdeck.com/dongjin
<https://speakerdeck.com/dongjin>*

Reply via email to