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

Reply via email to