Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2020-01-28 Thread Dongjin Lee
Hi Smiklos, Thanks for your interest in this issue. I am the author of this PR and now rebasing the code to the latest trunk. I have some questions: 1. Could you share how you conducted the benchmark? I want to run the full validation with all cases. 2. As you can see in the PR, it

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2020-01-14 Thread smiklos
Hi, Is there any update on this? I've done performance test with Avro data and Snappy compression. Setting the buffer from 32kb to 128kb brings a rough 10% decrease in storage which is a big deal. I could offer working on this as well. Best regards, Miklos

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-02-17 Thread Becket Qin
This thread has been quiet for some time now as there are a lot of things going on concurrently. @Ismael Juma @Jason Gustafson what do you think about this? Just a brief recap, we are considering exposing the configs for each compression type in the following k-v pairs format: compression.confi

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-02-07 Thread Dongjin Lee
Hi Becket, Understood. You are right, introducing map type is just a clearer way, but not a strong prerequisite. It can be discussed later. @Committers, PMC members: Could you please give us some valuable feedbacks and suggestions for this topic? Thanks, Dongjin On Fri, Feb 1, 2019 at 11:37

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-31 Thread Becket Qin
Hi Dongjin, Thanks for the reply. I agree that implementation wise, it is clearer to add a new "Map" config type. However, practically speaking I don't think this KIP strongly depends on that. As long as we follow the same string format in the configuration for all those map-like configurations, w

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-31 Thread Dongjin Lee
Mickael, It seems like the majority of the community agrees that the new config scheme proposed by Becket is much better. However, we still need another KIP to support this kind of config; it is different from the case of `listener.security.protocol.map,` which is just a concatenation of available

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-30 Thread Mickael Maison
Thanks, that's a very interesting KIP! I agree with Becket that a clearer config format is likely to help. Have you considered using a config format similar to listeners (as described in the "Updating SSL Keystore of an Existing Listener" in the Kafka docs)? Also worth noting that we already have

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-29 Thread Dongjin Lee
Hello. Do you have any idea on Becket's Idea of new config format (example below)? ``` compression.config="gzip.compression.level=5, lz4.compression.level=17, zstd.compression.level=22" ``` It requires some additional KIP for supporting new config format (map), but it can significantly simplify

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-24 Thread Dongjin Lee
Hi Becket, Thank you for your opinion. Frankly, I have no strong opinion on configuration name. In this problem, I will follow the community's choice. (I like your idea in that it has a notion of 'scope' per compression codec. However, it should be implemented on top of new config type like Map; I

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-22 Thread Becket Qin
Hi Dongjin, Thanks for the KIP and sorry for being a bit late on the discussion. It makes sense to expose the configuration for compression types. But I am wondering if there is a better way to do that than what proposed in the KIP. What I feel confusing is that we are effectively sharing the con

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-22 Thread Dongjin Lee
Hello. I just fixed the draft implementation, with rebasing onto the latest trunk. The KIP was also restored. Please have a look, and if there is no major problem, please vote to the voting thread. You know, KIP freeze for 2.2.0 is almost imminent. Thanks, Dongjin On Tue, Jan 22, 2019 at 1:04 AM

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-21 Thread Ismael Juma
Thanks! Ismael On Mon, Jan 21, 2019 at 6:02 AM Dongjin Lee wrote: > Hi Ismael, > > After reviewing `LogValidator#validateMessagesAndAssignOffsetsCompressed`, > yes, you are right. If source codec and target codec is identical and the > magic is above 0, the broker can do an in-place assignment,

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-21 Thread Dongjin Lee
Hi Ismael, After reviewing `LogValidator#validateMessagesAndAssignOffsetsCompressed`, yes, you are right. If source codec and target codec is identical and the magic is above 0, the broker can do an in-place assignment, without recompressing. Sorry for my misunderstanding. Since we don't need `co

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-20 Thread Dongjin Lee
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 wrote: > Hi Dongjin, > > Whe

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-20 Thread Ismael Juma
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

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-20 Thread Dongjin Lee
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 af

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-19 Thread Ismael Juma
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 wrote: > I just realize

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-19 Thread Dongjin Lee
I just updated the draft PR. Please have a look when you are free. Following the update, I have a minor update to the KIP document - 'compression.snappy.level' and 'compression.zstd.buffer.size' were removed since these options are meaningless. (Snappy doesn't support compression level, and zstd d

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-14 Thread Dongjin Lee
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

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-06 Thread Dongjin Lee
Thanks for pointing out Ismael. It's now updated. Best, Dongjin On Mon, Jan 7, 2019 at 4:36 AM Ismael Juma 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, 20

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-06 Thread Ismael Juma
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 Happy new year. > > I just updated the title and contents of KIP and Jira issue, with updated > draft

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2019-01-06 Thread Dongjin Lee
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/confl

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-12-02 Thread Ismael Juma
The updated title sounds fine to me. Ismael On Sun, Dec 2, 2018, 5:25 AM Dongjin Lee 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 > updat

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-12-02 Thread Dongjin Lee
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? @O

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-11-28 Thread Ismael Juma
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

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-11-28 Thread Dongjin Lee
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.

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-11-27 Thread Ismael Juma
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 some

Re: [DISCUSS] KIP-390: Add producer option to adjust compression level

2018-11-27 Thread Dongjin Lee
Is there anyone who is interested in this KIP? Thanks, Dongjin On Mon, Nov 19, 2018 at 7:06 AM Dongjin Lee wrote: > Hello dev, > > I hope to initiate the discussion of KIP-390: Add producer option to > adjust compression level >

[DISCUSS] KIP-390: Add producer option to adjust compression level

2018-11-18 Thread Dongjin Lee
Hello dev, I hope to initiate the discussion of KIP-390: Add producer option to adjust compression level . All feedbacks will be highly appreciated. Best, Dongjin -- *Dongjin Lee* *A h