Hi Dongjin,

The KIP looks good to me. I'd suggest starting a vote. A couple minor
points that might be worth calling out explicitly in the compatibility
section:

1. Zstd will only be allowed for the bumped produce API. For older
versions, we return UNSUPPORTED_COMPRESSION_TYPE regardless of the message
format.
2. Down-conversion of zstd-compressed records will not be supported.
Instead we will return UNSUPPORTED_COMPRESSION_TYPE.

Does that sound right?

Thanks,
Jason

On Thu, Sep 6, 2018 at 1:45 AM, Magnus Edenhill <mag...@edenhill.se> wrote:

> > Ismael wrote:
> > Jason, that's an interesting point regarding the Java client. Do we know
> > what clients in other languages do in these cases?
>
> librdkafka (and its bindings) passes unknown/future errors through to the
> application, the error code remains intact while
> the error string will be set to something like "Err-123?", which isn't very
> helpful to the user but it at least
> preserves the original error code for further troubleshooting.
> For the producer any unknown error returned in the ProduceResponse will be
> considered a permanent delivery failure (no retries),
> and for the consumer any unknown FetchResponse errors will propagate
> directly to the application, trigger a fetch backoff, and then
> continue fetching past that offset.
>
> So, from the client's perspective it is not really a problem if new error
> codes are added to older API versions.
>
> /Magnus
>
>
> Den tors 6 sep. 2018 kl 09:45 skrev Dongjin Lee <dong...@apache.org>:
>
> > I updated the KIP page
> > <
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> 110%3A+Add+Codec+for+ZStandard+Compression
> > >
> > following the discussion here. Please take a look when you are free.
> > If you have any opinion, don't hesitate to give me a message.
> >
> > Best,
> > Dongjin
> >
> > On Fri, Aug 31, 2018 at 11:35 PM Dongjin Lee <dong...@apache.org> wrote:
> >
> > > I just updated the draft implementation[^1], rebasing against the
> latest
> > > trunk and implementing error routine (i.e., Error code 74 for
> > > UnsupportedCompressionTypeException.) Since we decided to disallow all
> > > fetch request below version 2.1.0 for the topics specifying ZStandard,
> I
> > > added an error logic only.
> > >
> > > Please have a look when you are free.
> > >
> > > Thanks,
> > > Dongjin
> > >
> > > [^1]: Please check the last commit here:
> > > https://github.com/apache/kafka/pull/2267
> > >
> > > On Thu, Aug 23, 2018, 8:55 AM Dongjin Lee <dong...@apache.org> wrote:
> > >
> > >> Jason,
> > >>
> > >> Great. +1 for UNSUPPORTED_COMPRESSION_TYPE.
> > >>
> > >> Best,
> > >> Dongjin
> > >>
> > >> On Thu, Aug 23, 2018 at 8:19 AM Jason Gustafson <ja...@confluent.io>
> > >> wrote:
> > >>
> > >>> Hey Dongjin,
> > >>>
> > >>> Yeah that's right. For what it's worth, librdkafka also appears to
> > handle
> > >>> unexpected error codes. I expect that most client implementations
> would
> > >>> either pass through the raw type or convert to an enum using
> something
> > >>> like
> > >>> what the java client does. Since we're expecting the client to fail
> > >>> anyway,
> > >>> I'm probably in favor of using the UNSUPPORTED_COMPRESSION_TYPE error
> > >>> code.
> > >>>
> > >>> -Jason
> > >>>
> > >>> On Wed, Aug 22, 2018 at 1:46 AM, Dongjin Lee <dong...@apache.org>
> > wrote:
> > >>>
> > >>> > Jason and Ismael,
> > >>> >
> > >>> > It seems like the only thing we need to regard if we define a new
> > error
> > >>> > code (i.e., UNSUPPORTED_COMPRESSION_TYPE) would be the
> implementation
> > >>> of
> > >>> > the other language clients, right? At least, this strategy causes
> any
> > >>> > problem for Java client. Do I understand correctly?
> > >>> >
> > >>> > Thanks,
> > >>> > Dongjin
> > >>> >
> > >>> > On Wed, Aug 22, 2018 at 5:43 PM Dongjin Lee <dong...@apache.org>
> > >>> wrote:
> > >>> >
> > >>> > > Jason,
> > >>> > >
> > >>> > > > I think we would only use this error code when we /know/ that
> > zstd
> > >>> was
> > >>> > > in use and the client doesn't support it? This is true if either
> 1)
> > >>> the
> > >>> > > message needs down-conversion and we encounter a zstd compressed
> > >>> message,
> > >>> > > or 2) if the topic is explicitly configured to use zstd.
> > >>> > >
> > >>> > > Yes, it is right. And you know, the case 1 includes 1.a) old
> > clients'
> > >>> > > request v0, v1 records or 1.b) implicit zstd, the compression
> type
> > of
> > >>> > > "producer" with Zstd compressed data.
> > >>> > >
> > >>> > > > However, if the compression type is set to "producer," then the
> > >>> fetched
> > >>> > > data may or may not be compressed with zstd. In this case, we
> > return
> > >>> the
> > >>> > > data to the client and expect it to fail parsing. Is that
> correct?
> > >>> > >
> > >>> > > Exactly.
> > >>> > >
> > >>> > > Following your message, I reviewed the implementation of
> > >>> > > `KafkaApis#handleFetchRequest,` which handles the fetch request.
> > And
> > >>> > found
> > >>> > > that the information we can use is like the following:
> > >>> > >
> > >>> > > 1. Client's fetch request version. (`versionId` variable)
> > >>> > > 2. Log's compression type. (`logConfig` variable)
> > >>> > >
> > >>> > > We can't detect the actual compression type of the data, unless
> we
> > >>> > inspect
> > >>> > > the `RecordBatch` included in the `Records` instance (i.e.,
> > >>> > > `unconvertedRecords` variable.) Since it requires some
> performance
> > >>> issue,
> > >>> > > it is not our option - in short, we can't be sure if given chunks
> > of
> > >>> data
> > >>> > > are compressed with zstd or not.
> > >>> > >
> > >>> > > So, conclusion: we can return an error in the case of 1.a and 2
> > >>> easily,
> > >>> > > with the information above. In the case 1.b (implicit zstd), we
> can
> > >>> just
> > >>> > > return the data by do nothing special and expect it to fail
> > parsing.
> > >>> > >
> > >>> > > Thanks,
> > >>> > > Dongjin
> > >>> > >
> > >>> > > On Wed, Aug 22, 2018 at 12:02 PM Ismael Juma <isma...@gmail.com>
> > >>> wrote:
> > >>> > >
> > >>> > >> Jason, that's an interesting point regarding the Java client. Do
> > we
> > >>> know
> > >>> > >> what clients in other languages do in these cases?
> > >>> > >>
> > >>> > >> Ismael
> > >>> > >>
> > >>> > >> On Tue, 21 Aug 2018, 17:30 Jason Gustafson, <ja...@confluent.io
> >
> > >>> wrote:
> > >>> > >>
> > >>> > >> > Hi Dongjin,
> > >>> > >> >
> > >>> > >> > One of the complications is that old versions of the API will
> > not
> > >>> > >> expect a
> > >>> > >> > new error code. However, since we expect this to be a fatal
> > error
> > >>> > anyway
> > >>> > >> > for old clients, it may still be more useful to return the
> > correct
> > >>> > error
> > >>> > >> > code. For example, the Kafka clients use the following code to
> > >>> convert
> > >>> > >> the
> > >>> > >> > error code:
> > >>> > >> >
> > >>> > >> >     public static Errors forCode(short code) {
> > >>> > >> >         Errors error = codeToError.get(code);
> > >>> > >> >         if (error != null) {
> > >>> > >> >             return error;
> > >>> > >> >         } else {
> > >>> > >> >             log.warn("Unexpected error code: {}.", code);
> > >>> > >> >             return UNKNOWN_SERVER_ERROR;
> > >>> > >> >         }
> > >>> > >> >     }
> > >>> > >> >
> > >>> > >> > If we return an unsupported error code, it will be converted
> to
> > an
> > >>> > >> UNKNOWN
> > >>> > >> > error, but at least we will get the message in the log with
> the
> > >>> > correct
> > >>> > >> > code. That seems preferable to returning a misleading error
> > code.
> > >>> So I
> > >>> > >> > wonder if we can use the new UNSUPPORTED_COMPRESSION_TYPE
> error
> > >>> even
> > >>> > for
> > >>> > >> > older versions.
> > >>> > >> >
> > >>> > >> > Also, one question just to check my understanding. I think we
> > >>> would
> > >>> > only
> > >>> > >> > use this error code when we /know/ that zstd was in use and
> the
> > >>> client
> > >>> > >> > doesn't support it? This is true if either 1) the message
> needs
> > >>> > >> > down-conversion and we encounter a zstd compressed message, or
> > 2)
> > >>> if
> > >>> > the
> > >>> > >> > topic is explicitly configured to use zstd. However, if the
> > >>> > compression
> > >>> > >> > type is set to "producer," then the fetched data may or may
> not
> > be
> > >>> > >> > compressed with zstd. In this case, we return the data to the
> > >>> client
> > >>> > and
> > >>> > >> > expect it to fail parsing. Is that correct?
> > >>> > >> >
> > >>> > >> > Thanks,
> > >>> > >> > Jason
> > >>> > >> >
> > >>> > >> >
> > >>> > >> >
> > >>> > >> > On Tue, Aug 21, 2018 at 9:08 AM, Dongjin Lee <
> > dong...@apache.org>
> > >>> > >> wrote:
> > >>> > >> >
> > >>> > >> > > Ismael, Jason and all,
> > >>> > >> > >
> > >>> > >> > > I rewrote the backward compatibility strategy & its
> > alternatives
> > >>> > like
> > >>> > >> > > following, based on Ismael & Jason's comments. Since it is
> not
> > >>> > >> updated to
> > >>> > >> > > the wiki yet, don't hesitate to give me a message if you
> have
> > >>> any
> > >>> > >> opinion
> > >>> > >> > > on it.
> > >>> > >> > >
> > >>> > >> > > ```
> > >>> > >> > > *Backward Compatibility*
> > >>> > >> > >
> > >>> > >> > > We need to establish some backward-compatibility strategy
> for
> > >>> the
> > >>> > >> case an
> > >>> > >> > > old client subscribes a topic using ZStandard implicitly
> > (i.e.,
> > >>> > >> > > 'compression.type' configuration of given topic is
> 'producer'
> > >>> and
> > >>> > the
> > >>> > >> > > producer compressed the records with ZStandard). We have the
> > >>> > following
> > >>> > >> > > options for this situation:
> > >>> > >> > >
> > >>> > >> > > *A. Support ZStandard to the old clients which can
> understand
> > >>> v0, v1
> > >>> > >> > > messages only.*
> > >>> > >> > >
> > >>> > >> > > This strategy necessarily requires the down-conversion of v2
> > >>> message
> > >>> > >> > > compressed with Zstandard into v0 or v1 messages, which
> means
> > a
> > >>> > >> > > considerable performance degradation. So we rejected this
> > >>> strategy.
> > >>> > >> > >
> > >>> > >> > > *B. Bump the API version and support only v2-available
> > clients*
> > >>> > >> > >
> > >>> > >> > > With this approach, we can message the old clients that they
> > >>> are old
> > >>> > >> and
> > >>> > >> > > should be upgraded. However, there are still several options
> > >>> for the
> > >>> > >> > Error
> > >>> > >> > > code.
> > >>> > >> > >
> > >>> > >> > > *B.1. INVALID_REQUEST (42)*
> > >>> > >> > >
> > >>> > >> > > This option gives the client so little information; the user
> > >>> can be
> > >>> > >> > > confused about why the client worked correctly in the past
> > >>> suddenly
> > >>> > >> > > encounters a problem. So we rejected this strategy.
> > >>> > >> > >
> > >>> > >> > > *B.2. CORRUPT_MESSAGE (2)*
> > >>> > >> > >
> > >>> > >> > > This option gives inaccurate information; the user can be
> > >>> surprised
> > >>> > >> and
> > >>> > >> > > misunderstand that the log files are broken in some way. So
> we
> > >>> > >> rejected
> > >>> > >> > > this strategy.
> > >>> > >> > >
> > >>> > >> > > *B.3 UNSUPPORTED_FOR_MESSAGE_FORMAT (43)*
> > >>> > >> > >
> > >>> > >> > > The advantage of this approach is we don't need to define a
> > new
> > >>> > error
> > >>> > >> > code;
> > >>> > >> > > we can reuse it and that's all.
> > >>> > >> > >
> > >>> > >> > > The disadvantage of this approach is that it is also a
> little
> > >>> bit
> > >>> > >> vague;
> > >>> > >> > > This error code is defined as a work for KIP-98[^1] and now
> > >>> returned
> > >>> > >> in
> > >>> > >> > the
> > >>> > >> > > transaction error.
> > >>> > >> > >
> > >>> > >> > > *B.4. UNSUPPORTED_COMPRESSION_TYPE (new)*
> > >>> > >> > >
> > >>> > >> > > The advantage of this approach is that it is clear and
> > provides
> > >>> an
> > >>> > >> exact
> > >>> > >> > > description. The disadvantage is we need to add a new error
> > >>> code.
> > >>> > >> > > ```
> > >>> > >> > >
> > >>> > >> > > *It seems like what we need to choose is now so clear:
> > >>> > >> > > UNSUPPORTED_FOR_MESSAGE_FORMAT (B.3) or
> > >>> UNSUPPORTED_COMPRESSION_TYPE
> > >>> > >> > > (B.4).*
> > >>> > >> > > The first one doesn't need a new error message but the
> latter
> > is
> > >>> > more
> > >>> > >> > > explicit. Which one do you prefer? Since all of you have
> much
> > >>> more
> > >>> > >> > > experience and knowledge than me, I will follow your
> decision.
> > >>> The
> > >>> > >> wiki
> > >>> > >> > > page will be updated following the decision also.
> > >>> > >> > >
> > >>> > >> > > Best,
> > >>> > >> > > Dongjin
> > >>> > >> > >
> > >>> > >> > > [^1]: https://issues.apache.org/jira/browse/KAFKA-4990
> > >>> > >> > >
> > >>> > >> > > On Sun, Aug 19, 2018 at 4:58 AM Ismael Juma <
> > isma...@gmail.com>
> > >>> > >> wrote:
> > >>> > >> > >
> > >>> > >> > > > Sounds reasonable to me.
> > >>> > >> > > >
> > >>> > >> > > > Ismael
> > >>> > >> > > >
> > >>> > >> > > > On Sat, 18 Aug 2018, 12:20 Jason Gustafson, <
> > >>> ja...@confluent.io>
> > >>> > >> > wrote:
> > >>> > >> > > >
> > >>> > >> > > > > Hey Ismael,
> > >>> > >> > > > >
> > >>> > >> > > > > Your summary looks good to me. I think it might also be
> a
> > >>> good
> > >>> > >> idea
> > >>> > >> > to
> > >>> > >> > > > add
> > >>> > >> > > > > a new UNSUPPORTED_COMPRESSION_TYPE error code to go
> along
> > >>> with
> > >>> > the
> > >>> > >> > > > version
> > >>> > >> > > > > bumps. We won't be able to use it for old api versions
> > >>> since the
> > >>> > >> > > clients
> > >>> > >> > > > > will not understand it, but we can use it going forward
> so
> > >>> that
> > >>> > >> we're
> > >>> > >> > > not
> > >>> > >> > > > > stuck in a similar situation with a new message format
> and
> > >>> a new
> > >>> > >> > codec
> > >>> > >> > > to
> > >>> > >> > > > > support. Another option is to use
> UNSUPPORTED_FOR_MESSAGE_
> > >>> > FORMAT,
> > >>> > >> but
> > >>> > >> > > it
> > >>> > >> > > > is
> > >>> > >> > > > > not as explicit.
> > >>> > >> > > > >
> > >>> > >> > > > > -Jason
> > >>> > >> > > > >
> > >>> > >> > > > > On Fri, Aug 17, 2018 at 5:19 PM, Ismael Juma <
> > >>> ism...@juma.me.uk
> > >>> > >
> > >>> > >> > > wrote:
> > >>> > >> > > > >
> > >>> > >> > > > > > Hi Dongjin and Jason,
> > >>> > >> > > > > >
> > >>> > >> > > > > > I would agree. My summary:
> > >>> > >> > > > > >
> > >>> > >> > > > > > 1. Support zstd with message format 2 only.
> > >>> > >> > > > > > 2. Bump produce and fetch request versions.
> > >>> > >> > > > > > 3. Provide broker errors whenever possible based on
> the
> > >>> > request
> > >>> > >> > > version
> > >>> > >> > > > > and
> > >>> > >> > > > > > rely on clients for the cases where the broker can't
> > >>> validate
> > >>> > >> > > > efficiently
> > >>> > >> > > > > > (example message format 2 consumer that supports the
> > >>> latest
> > >>> > >> fetch
> > >>> > >> > > > version
> > >>> > >> > > > > > but doesn't support zstd).
> > >>> > >> > > > > >
> > >>> > >> > > > > > If there's general agreement on this, I suggest we
> > update
> > >>> the
> > >>> > >> KIP
> > >>> > >> > to
> > >>> > >> > > > > state
> > >>> > >> > > > > > the proposal and to move the rejected options to its
> own
> > >>> > >> section.
> > >>> > >> > And
> > >>> > >> > > > > then
> > >>> > >> > > > > > start a vote!
> > >>> > >> > > > > >
> > >>> > >> > > > > > Ismael
> > >>> > >> > > > > >
> > >>> > >> > > > > > On Fri, Aug 17, 2018 at 4:00 PM Jason Gustafson <
> > >>> > >> > ja...@confluent.io>
> > >>> > >> > > > > > wrote:
> > >>> > >> > > > > >
> > >>> > >> > > > > > > Hi Dongjin,
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > Yes, that's a good summary. For clients which
> support
> > >>> v2,
> > >>> > the
> > >>> > >> > > client
> > >>> > >> > > > > can
> > >>> > >> > > > > > > parse the message format and hopefully raise a
> useful
> > >>> error
> > >>> > >> > message
> > >>> > >> > > > > > > indicating the unsupported compression type. For
> older
> > >>> > >> clients,
> > >>> > >> > our
> > >>> > >> > > > > > options
> > >>> > >> > > > > > > are probably (1) to down-convert to the old format
> > >>> using no
> > >>> > >> > > > compression
> > >>> > >> > > > > > > type, or (2) to return an error code. I'm leaning
> > >>> toward the
> > >>> > >> > latter
> > >>> > >> > > > as
> > >>> > >> > > > > > the
> > >>> > >> > > > > > > simpler solution, but the challenge is finding a
> good
> > >>> error
> > >>> > >> code.
> > >>> > >> > > Two
> > >>> > >> > > > > > > possibilities might be INVALID_REQUEST or
> > >>> CORRUPT_MESSAGE.
> > >>> > The
> > >>> > >> > > > downside
> > >>> > >> > > > > > is
> > >>> > >> > > > > > > that old clients probably won't get a helpful
> message.
> > >>> > >> However,
> > >>> > >> > at
> > >>> > >> > > > > least
> > >>> > >> > > > > > > the behavior will be consistent in the sense that
> all
> > >>> > clients
> > >>> > >> > will
> > >>> > >> > > > fail
> > >>> > >> > > > > > if
> > >>> > >> > > > > > > they do not support zstandard.
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > What do you think?
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > Thanks,
> > >>> > >> > > > > > > Jason
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > On Fri, Aug 17, 2018 at 8:08 AM, Dongjin Lee <
> > >>> > >> dong...@apache.org
> > >>> > >> > >
> > >>> > >> > > > > wrote:
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > > Thanks Jason, I reviewed the down-converting logic
> > >>> > following
> > >>> > >> > your
> > >>> > >> > > > > > > > explanation.[^1] You mean the following routines,
> > >>> right?
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > -
> > >>> > >> > > > > > > >
> > https://github.com/apache/kafka/blob/trunk/core/src/
> > >>> > >> > > > > > > > main/scala/kafka/server/KafkaApis.scala#L534
> > >>> > >> > > > > > > > -
> > >>> > >> > > > > > > >
> > >>> https://github.com/apache/kafka/blob/trunk/clients/src/
> > >>> > >> > > > > > > > main/java/org/apache/kafka/common/record/
> > >>> > >> > > LazyDownConversionRecords.
> > >>> > >> > > > > > > > java#L165
> > >>> > >> > > > > > > > -
> > >>> > >> > > > > > > >
> > >>> https://github.com/apache/kafka/blob/trunk/clients/src/
> > >>> > >> > > > > > > >
> > >>> > >> main/java/org/apache/kafka/common/record/RecordsUtil.java#L40
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > It seems like your stance is like following:
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > 1. In principle, Kafka does not change the
> > compression
> > >>> > codec
> > >>> > >> > when
> > >>> > >> > > > > > > > down-converting, since it requires inspecting the
> > >>> fetched
> > >>> > >> data,
> > >>> > >> > > > which
> > >>> > >> > > > > > is
> > >>> > >> > > > > > > > expensive.
> > >>> > >> > > > > > > > 2. However, there are some cases the fetched data
> is
> > >>> > >> inspected
> > >>> > >> > > > > anyway.
> > >>> > >> > > > > > In
> > >>> > >> > > > > > > > this case, we can provide compression conversion
> > from
> > >>> > >> Zstandard
> > >>> > >> > > to
> > >>> > >> > > > > > > > classical ones[^2].
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > And from what I understand, the cases where the
> > client
> > >>> > >> without
> > >>> > >> > > > > > ZStandard
> > >>> > >> > > > > > > > support receives ZStandard compressed records can
> be
> > >>> > >> organized
> > >>> > >> > > into
> > >>> > >> > > > > two
> > >>> > >> > > > > > > > cases:
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > a. The 'compression.type' configuration of given
> > >>> topic is
> > >>> > >> > > > 'producer'
> > >>> > >> > > > > > and
> > >>> > >> > > > > > > > the producer compressed the records with
> ZStandard.
> > >>> (that
> > >>> > >> is,
> > >>> > >> > > using
> > >>> > >> > > > > > > > ZStandard implicitly.)
> > >>> > >> > > > > > > > b.  The 'compression.type' configuration of given
> > >>> topic is
> > >>> > >> > > 'zstd';
> > >>> > >> > > > > that
> > >>> > >> > > > > > > is,
> > >>> > >> > > > > > > > using ZStandard explicitly.
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > As you stated, we don't have to handle the case b
> > >>> > specially.
> > >>> > >> > So,
> > >>> > >> > > It
> > >>> > >> > > > > > seems
> > >>> > >> > > > > > > > like we can narrow the focus of the problem by
> > joining
> > >>> > case
> > >>> > >> 1
> > >>> > >> > and
> > >>> > >> > > > > case
> > >>> > >> > > > > > b
> > >>> > >> > > > > > > > like the following:
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > > Given the topic with 'producer' as its
> > >>> > 'compression.type'
> > >>> > >> > > > > > > configuration,
> > >>> > >> > > > > > > > ZStandard compressed records and old client
> without
> > >>> > >> ZStandard,
> > >>> > >> > is
> > >>> > >> > > > > there
> > >>> > >> > > > > > > any
> > >>> > >> > > > > > > > case we need to inspect the records and can change
> > the
> > >>> > >> > > compression
> > >>> > >> > > > > > type?
> > >>> > >> > > > > > > If
> > >>> > >> > > > > > > > so, can we provide compression type converting?
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > Do I understand correctly?
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > Best,
> > >>> > >> > > > > > > > Dongjin
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > [^1]: I'm sorry, I found that I was a little bit
> > >>> > >> > misunderstanding
> > >>> > >> > > > how
> > >>> > >> > > > > > API
> > >>> > >> > > > > > > > version works, after reviewing the downconvert
> logic
> > >>> & the
> > >>> > >> > > protocol
> > >>> > >> > > > > > > > documentation <https://kafka.apache.org/protocol
> >.
> > >>> > >> > > > > > > > [^2]: None, Gzip, Snappy, Lz4.
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > On Tue, Aug 14, 2018 at 2:16 AM Jason Gustafson <
> > >>> > >> > > > ja...@confluent.io>
> > >>> > >> > > > > > > > wrote:
> > >>> > >> > > > > > > >
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > But in my opinion, since the client will fail
> > >>> with the
> > >>> > >> API
> > >>> > >> > > > > version,
> > >>> > >> > > > > > > so
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > > don't need to down-convert the messages
> anyway.
> > >>> Isn't
> > >>> > >> it?
> > >>> > >> > > So, I
> > >>> > >> > > > > > think
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > > don't care about this case. (I'm sorry, I am
> not
> > >>> > >> familiar
> > >>> > >> > > with
> > >>> > >> > > > > > > > > down-convert
> > >>> > >> > > > > > > > > > logic.)
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > Currently the broker down-converts automatically
> > >>> when it
> > >>> > >> > > receives
> > >>> > >> > > > > an
> > >>> > >> > > > > > > old
> > >>> > >> > > > > > > > > version of the fetch request (a version which is
> > >>> known
> > >>> > to
> > >>> > >> > > predate
> > >>> > >> > > > > the
> > >>> > >> > > > > > > > > message format in use). Typically when
> > >>> down-converting
> > >>> > the
> > >>> > >> > > > message
> > >>> > >> > > > > > > > format,
> > >>> > >> > > > > > > > > we use the same compression type, but there is
> not
> > >>> much
> > >>> > >> point
> > >>> > >> > > in
> > >>> > >> > > > > > doing
> > >>> > >> > > > > > > so
> > >>> > >> > > > > > > > > when we know the client doesn't support it. So
> if
> > >>> > >> zstandard
> > >>> > >> > is
> > >>> > >> > > in
> > >>> > >> > > > > > use,
> > >>> > >> > > > > > > > and
> > >>> > >> > > > > > > > > we have to down-convert anyway, then we can
> choose
> > >>> to
> > >>> > use
> > >>> > >> a
> > >>> > >> > > > > different
> > >>> > >> > > > > > > > > compression type or no compression type.
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > From my perspective, there is no significant
> > >>> downside to
> > >>> > >> > > bumping
> > >>> > >> > > > > the
> > >>> > >> > > > > > > > > protocol version and it has several potential
> > >>> benefits.
> > >>> > >> > Version
> > >>> > >> > > > > bumps
> > >>> > >> > > > > > > are
> > >>> > >> > > > > > > > > cheap. The main question mark in my mind is
> about
> > >>> > >> > > > down-conversion.
> > >>> > >> > > > > > > > Figuring
> > >>> > >> > > > > > > > > out whether down-conversion is needed is hard
> > >>> generally
> > >>> > >> > without
> > >>> > >> > > > > > > > inspecting
> > >>> > >> > > > > > > > > the fetched data, which is expensive. I think we
> > >>> agree
> > >>> > in
> > >>> > >> > > > principle
> > >>> > >> > > > > > > that
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > do not want to have to pay this cost generally
> and
> > >>> > prefer
> > >>> > >> the
> > >>> > >> > > > > clients
> > >>> > >> > > > > > > to
> > >>> > >> > > > > > > > > fail when they see an unhandled compression
> type.
> > >>> The
> > >>> > >> point I
> > >>> > >> > > was
> > >>> > >> > > > > > > making
> > >>> > >> > > > > > > > is
> > >>> > >> > > > > > > > > that there are some cases where we are either
> > >>> inspecting
> > >>> > >> the
> > >>> > >> > > data
> > >>> > >> > > > > > > anyway
> > >>> > >> > > > > > > > > (because we have to down-convert the message
> > >>> format), or
> > >>> > >> we
> > >>> > >> > > have
> > >>> > >> > > > an
> > >>> > >> > > > > > > easy
> > >>> > >> > > > > > > > > way to tell whether zstandard is in use (the
> topic
> > >>> has
> > >>> > it
> > >>> > >> > > > > configured
> > >>> > >> > > > > > > > > explicitly). In the latter case, we don't have
> to
> > >>> handle
> > >>> > >> it
> > >>> > >> > > > > > specially.
> > >>> > >> > > > > > > > But
> > >>> > >> > > > > > > > > we do have to decide how we will handle
> > >>> down-conversion
> > >>> > to
> > >>> > >> > > older
> > >>> > >> > > > > > > formats.
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > -Jason
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > On Sun, Aug 12, 2018 at 5:15 PM, Dongjin Lee <
> > >>> > >> > > dong...@apache.org
> > >>> > >> > > > >
> > >>> > >> > > > > > > wrote:
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > > Colin and Jason,
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > Thanks for your opinions. In summarizing, the
> > >>> Pros and
> > >>> > >> Cons
> > >>> > >> > > of
> > >>> > >> > > > > > > bumping
> > >>> > >> > > > > > > > > > fetch API version are:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > Cons:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > - The Broker can't know whether a given
> message
> > >>> batch
> > >>> > is
> > >>> > >> > > > > compressed
> > >>> > >> > > > > > > > with
> > >>> > >> > > > > > > > > > zstd or not.
> > >>> > >> > > > > > > > > > - Need some additional logic for the topic
> > >>> explicitly
> > >>> > >> > > > configured
> > >>> > >> > > > > to
> > >>> > >> > > > > > > use
> > >>> > >> > > > > > > > > > zstd.
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > Pros:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > - The broker doesn't need to conduct expensive
> > >>> > >> > > down-conversion.
> > >>> > >> > > > > > > > > > - Can message the users to update their
> client.
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > So, opinions for the backward-compatibility
> > >>> policy by
> > >>> > >> far:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > - A: bump the API version - +2 (Colin, Jason)
> > >>> > >> > > > > > > > > > - B: leave unchanged - +1 (Viktor)
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > Here are my additional comments:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > @Colin
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > I greatly appreciate your response. In the
> case
> > >>> of the
> > >>> > >> > > > dictionary
> > >>> > >> > > > > > > > > support,
> > >>> > >> > > > > > > > > > of course, this issue should be addressed
> later
> > >>> so we
> > >>> > >> don't
> > >>> > >> > > > need
> > >>> > >> > > > > it
> > >>> > >> > > > > > > in
> > >>> > >> > > > > > > > > the
> > >>> > >> > > > > > > > > > first version. You are right - it is not late
> to
> > >>> try
> > >>> > it
> > >>> > >> > after
> > >>> > >> > > > > some
> > >>> > >> > > > > > > > > > benchmarks. What I mean is, we should keep in
> > >>> mind on
> > >>> > >> that
> > >>> > >> > > > > > potential
> > >>> > >> > > > > > > > > > feature.
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > @Jason
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > You wrote,
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > > Similarly, if we have to down-convert anyway
> > >>> because
> > >>> > >> the
> > >>> > >> > > > client
> > >>> > >> > > > > > > does
> > >>> > >> > > > > > > > > not
> > >>> > >> > > > > > > > > > understand the message format, then we could
> > also
> > >>> use
> > >>> > a
> > >>> > >> > > > different
> > >>> > >> > > > > > > > > > compression type.
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > But in my opinion, since the client will fail
> > >>> with the
> > >>> > >> API
> > >>> > >> > > > > version,
> > >>> > >> > > > > > > so
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > > don't need to down-convert the messages
> anyway.
> > >>> Isn't
> > >>> > >> it?
> > >>> > >> > > So, I
> > >>> > >> > > > > > think
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > > don't care about this case. (I'm sorry, I am
> not
> > >>> > >> familiar
> > >>> > >> > > with
> > >>> > >> > > > > > > > > down-convert
> > >>> > >> > > > > > > > > > logic.)
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > Please give more opinions. Thanks!
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > - Dongjin
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > On Wed, Aug 8, 2018 at 6:41 AM Jason
> Gustafson <
> > >>> > >> > > > > ja...@confluent.io
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > > > wrote:
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > > Hey Colin,
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > The problem for the fetch API is that the
> > broker
> > >>> > does
> > >>> > >> not
> > >>> > >> > > > > > generally
> > >>> > >> > > > > > > > > know
> > >>> > >> > > > > > > > > > if
> > >>> > >> > > > > > > > > > > a batch was compressed with zstd unless it
> > >>> parses
> > >>> > it.
> > >>> > >> I
> > >>> > >> > > think
> > >>> > >> > > > > the
> > >>> > >> > > > > > > > goal
> > >>> > >> > > > > > > > > > here
> > >>> > >> > > > > > > > > > > is to avoid the expensive down-conversion
> that
> > >>> is
> > >>> > >> needed
> > >>> > >> > to
> > >>> > >> > > > > > ensure
> > >>> > >> > > > > > > > > > > compatibility because it is only necessary
> if
> > >>> zstd
> > >>> > is
> > >>> > >> > > > actually
> > >>> > >> > > > > in
> > >>> > >> > > > > > > > use.
> > >>> > >> > > > > > > > > > But
> > >>> > >> > > > > > > > > > > as long as old clients can parse the message
> > >>> format,
> > >>> > >> they
> > >>> > >> > > > > should
> > >>> > >> > > > > > > get
> > >>> > >> > > > > > > > a
> > >>> > >> > > > > > > > > > > reasonable error if they see an unsupported
> > >>> > >> compression
> > >>> > >> > > type
> > >>> > >> > > > in
> > >>> > >> > > > > > the
> > >>> > >> > > > > > > > > > > attributes. Basically the onus is on users
> to
> > >>> ensure
> > >>> > >> that
> > >>> > >> > > > their
> > >>> > >> > > > > > > > > consumers
> > >>> > >> > > > > > > > > > > have been updated prior to using zstd. It
> > seems
> > >>> > like a
> > >>> > >> > > > > reasonable
> > >>> > >> > > > > > > > > > tradeoff
> > >>> > >> > > > > > > > > > > to me. There are a couple cases that might
> be
> > >>> worth
> > >>> > >> > > thinking
> > >>> > >> > > > > > > through:
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > 1. If a topic is explicitly configured to
> use
> > >>> zstd,
> > >>> > >> then
> > >>> > >> > we
> > >>> > >> > > > > don't
> > >>> > >> > > > > > > > need
> > >>> > >> > > > > > > > > to
> > >>> > >> > > > > > > > > > > check the fetched data for the compression
> > type
> > >>> to
> > >>> > >> know
> > >>> > >> > if
> > >>> > >> > > we
> > >>> > >> > > > > > need
> > >>> > >> > > > > > > > > > > down-conversion. If we did bump the Fetch
> API
> > >>> > version,
> > >>> > >> > then
> > >>> > >> > > > we
> > >>> > >> > > > > > > could
> > >>> > >> > > > > > > > > > handle
> > >>> > >> > > > > > > > > > > this case by either down-converting using a
> > >>> > different
> > >>> > >> > > > > compression
> > >>> > >> > > > > > > > type
> > >>> > >> > > > > > > > > or
> > >>> > >> > > > > > > > > > > returning an error.
> > >>> > >> > > > > > > > > > > 2. Similarly, if we have to down-convert
> > anyway
> > >>> > >> because
> > >>> > >> > the
> > >>> > >> > > > > > client
> > >>> > >> > > > > > > > does
> > >>> > >> > > > > > > > > > not
> > >>> > >> > > > > > > > > > > understand the message format, then we could
> > >>> also
> > >>> > use
> > >>> > >> a
> > >>> > >> > > > > different
> > >>> > >> > > > > > > > > > > compression type.
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > For the produce API, I think it's reasonable
> > to
> > >>> bump
> > >>> > >> the
> > >>> > >> > > api
> > >>> > >> > > > > > > version.
> > >>> > >> > > > > > > > > > This
> > >>> > >> > > > > > > > > > > can be used by clients to check whether a
> > broker
> > >>> > >> supports
> > >>> > >> > > > zstd.
> > >>> > >> > > > > > For
> > >>> > >> > > > > > > > > > > example, we might support a list of
> preferred
> > >>> > >> compression
> > >>> > >> > > > types
> > >>> > >> > > > > > in
> > >>> > >> > > > > > > > the
> > >>> > >> > > > > > > > > > > producer and we could use the broker to
> detect
> > >>> which
> > >>> > >> > > version
> > >>> > >> > > > to
> > >>> > >> > > > > > > use.
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > -Jason
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > On Tue, Aug 7, 2018 at 1:32 PM, Colin
> McCabe <
> > >>> > >> > > > > cmcc...@apache.org
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > Thanks for bumping this, Dongjin.  ZStd
> is a
> > >>> good
> > >>> > >> > > > compression
> > >>> > >> > > > > > > codec
> > >>> > >> > > > > > > > > > and I
> > >>> > >> > > > > > > > > > > > hope we can get this support in soon!
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > I would say we can just bump the API
> version
> > >>> to
> > >>> > >> > indicate
> > >>> > >> > > > that
> > >>> > >> > > > > > > ZStd
> > >>> > >> > > > > > > > > > > support
> > >>> > >> > > > > > > > > > > > is expected in new clients.  We probably
> > need
> > >>> some
> > >>> > >> way
> > >>> > >> > of
> > >>> > >> > > > > > > > indicating
> > >>> > >> > > > > > > > > to
> > >>> > >> > > > > > > > > > > the
> > >>> > >> > > > > > > > > > > > older clients that they can't consume the
> > >>> > >> partitions,
> > >>> > >> > as
> > >>> > >> > > > > well.
> > >>> > >> > > > > > > > > Perhaps
> > >>> > >> > > > > > > > > > > we
> > >>> > >> > > > > > > > > > > > can use the UNSUPPORTED_FOR_MESSAGE_FORMAT
> > >>> error?
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > The license thing seems straightforward --
> > >>> it's
> > >>> > >> just a
> > >>> > >> > > > matter
> > >>> > >> > > > > > of
> > >>> > >> > > > > > > > > adding
> > >>> > >> > > > > > > > > > > > the text to the right files as per ASF
> > >>> guidelines.
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > With regard to the dictionary support, do
> we
> > >>> > really
> > >>> > >> > need
> > >>> > >> > > > that
> > >>> > >> > > > > > in
> > >>> > >> > > > > > > > the
> > >>> > >> > > > > > > > > > > first
> > >>> > >> > > > > > > > > > > > version?  Hopefully message batches are
> big
> > >>> enough
> > >>> > >> that
> > >>> > >> > > > this
> > >>> > >> > > > > > > isn't
> > >>> > >> > > > > > > > > > > needed.
> > >>> > >> > > > > > > > > > > > Some benchmarks might help here.
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > best,
> > >>> > >> > > > > > > > > > > > Colin
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > On Tue, Aug 7, 2018, at 08:02, Dongjin Lee
> > >>> wrote:
> > >>> > >> > > > > > > > > > > > > As Kafka 2.0.0 was released, let's
> reboot
> > >>> this
> > >>> > >> issue,
> > >>> > >> > > > > KIP-110
> > >>> > >> > > > > > > > > > > > > <https://cwiki.apache.org/
> > >>> > >> > > confluence/display/KAFKA/KIP-
> > >>> > >> > > > > > > > > > > > 110%3A+Add+Codec+for+
> ZStandard+Compression>
> > >>> > >> > > > > > > > > > > > > .
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > For newcomers, Here is some summary of
> the
> > >>> > >> history:
> > >>> > >> > > > KIP-110
> > >>> > >> > > > > > was
> > >>> > >> > > > > > > > > > > > originally
> > >>> > >> > > > > > > > > > > > > worked for the issue KAFKA-4514 but, it
> > >>> lacked
> > >>> > >> > > benchmark
> > >>> > >> > > > > > > results
> > >>> > >> > > > > > > > to
> > >>> > >> > > > > > > > > > get
> > >>> > >> > > > > > > > > > > > the
> > >>> > >> > > > > > > > > > > > > agreement of the community. Later, Ivan
> > >>> Babrou
> > >>> > and
> > >>> > >> > some
> > >>> > >> > > > > other
> > >>> > >> > > > > > > > users
> > >>> > >> > > > > > > > > > who
> > >>> > >> > > > > > > > > > > > > adopted the patch provided their
> excellent
> > >>> > >> > performance
> > >>> > >> > > > > report
> > >>> > >> > > > > > > > which
> > >>> > >> > > > > > > > > > is
> > >>> > >> > > > > > > > > > > > now
> > >>> > >> > > > > > > > > > > > > included in the KIP, but it postponed
> > again
> > >>> > >> because
> > >>> > >> > of
> > >>> > >> > > > the
> > >>> > >> > > > > > > > > community
> > >>> > >> > > > > > > > > > > was
> > >>> > >> > > > > > > > > > > > > busy for 2.0.0 release. It is why I now
> > >>> reboot
> > >>> > >> this
> > >>> > >> > > > issue.
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > The following is the current status of
> the
> > >>> > >> feature:
> > >>> > >> > You
> > >>> > >> > > > can
> > >>> > >> > > > > > > check
> > >>> > >> > > > > > > > > the
> > >>> > >> > > > > > > > > > > > > current draft implementation here
> > >>> > >> > > > > > > > > > > > > <
> > https://github.com/apache/kafka/pull/2267>.
> > >>> It
> > >>> > >> is
> > >>> > >> > > based
> > >>> > >> > > > > on
> > >>> > >> > > > > > > zstd
> > >>> > >> > > > > > > > > > 1.3.5
> > >>> > >> > > > > > > > > > > > and
> > >>> > >> > > > > > > > > > > > > periodically rebased onto the latest
> > >>> trunk[^1].
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > The issues that should be addressed is
> > like
> > >>> > >> > following:
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > *1. Backward Compatibility*
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > To support old consumers, we need to
> take
> > a
> > >>> > >> strategy
> > >>> > >> > to
> > >>> > >> > > > > > handle
> > >>> > >> > > > > > > > the
> > >>> > >> > > > > > > > > > old
> > >>> > >> > > > > > > > > > > > > consumers. Current candidates are:
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > - Bump API version
> > >>> > >> > > > > > > > > > > > > - Leave unchanged: let the old clients
> > fail.
> > >>> > >> > > > > > > > > > > > > - Improve the error messages:
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > *2. Dictionary Support*
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > To support zstd's dictionary feature in
> > the
> > >>> > future
> > >>> > >> > (if
> > >>> > >> > > > > > needed),
> > >>> > >> > > > > > > > we
> > >>> > >> > > > > > > > > > need
> > >>> > >> > > > > > > > > > > > to
> > >>> > >> > > > > > > > > > > > > sketch how it should be and leave some
> > room
> > >>> for
> > >>> > >> it.
> > >>> > >> > As
> > >>> > >> > > of
> > >>> > >> > > > > > now,
> > >>> > >> > > > > > > > > there
> > >>> > >> > > > > > > > > > > has
> > >>> > >> > > > > > > > > > > > > been no discussion on this topic yet.
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > *3. License*
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > To use this feature, we need to add
> > license
> > >>> of
> > >>> > >> zstd
> > >>> > >> > and
> > >>> > >> > > > > > > zstd-jni
> > >>> > >> > > > > > > > to
> > >>> > >> > > > > > > > > > the
> > >>> > >> > > > > > > > > > > > > project. (Thanks to Viktor Somogyi for
> > >>> raising
> > >>> > >> this
> > >>> > >> > > > issue!)
> > >>> > >> > > > > > It
> > >>> > >> > > > > > > > > seems
> > >>> > >> > > > > > > > > > > like
> > >>> > >> > > > > > > > > > > > > what Apache Spark did would be a good
> > >>> example
> > >>> > but
> > >>> > >> > there
> > >>> > >> > > > has
> > >>> > >> > > > > > > been
> > >>> > >> > > > > > > > no
> > >>> > >> > > > > > > > > > > > > discussion yet.
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > You can find the details of the above
> > >>> issues in
> > >>> > >> the
> > >>> > >> > KIP
> > >>> > >> > > > > > > document.
> > >>> > >> > > > > > > > > > > Please
> > >>> > >> > > > > > > > > > > > > have a look when you are free, and give
> me
> > >>> > >> feedback.
> > >>> > >> > > All
> > >>> > >> > > > > > kinds
> > >>> > >> > > > > > > of
> > >>> > >> > > > > > > > > > > > > participating are welcome.
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > Best,
> > >>> > >> > > > > > > > > > > > > Dongjin
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > [^1]: At the time of writing, commit
> > >>> 6b4fb8152.
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > On Sat, Jul 14, 2018 at 10:45 PM Dongjin
> > >>> Lee <
> > >>> > >> > > > > > > dong...@apache.org
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > > Sorry for the late reply.
> > >>> > >> > > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > > In short, I could not submit the
> updated
> > >>> KIP
> > >>> > by
> > >>> > >> the
> > >>> > >> > > > > feature
> > >>> > >> > > > > > > > > freeze
> > >>> > >> > > > > > > > > > > > > > deadline of 2.0.0. For this reason, it
> > >>> will
> > >>> > not
> > >>> > >> be
> > >>> > >> > > > > included
> > >>> > >> > > > > > > in
> > >>> > >> > > > > > > > > the
> > >>> > >> > > > > > > > > > > > 2.0.0
> > >>> > >> > > > > > > > > > > > > > release and all discussion for this
> > issue
> > >>> were
> > >>> > >> > > > postponed
> > >>> > >> > > > > > > after
> > >>> > >> > > > > > > > > the
> > >>> > >> > > > > > > > > > > > release
> > >>> > >> > > > > > > > > > > > > > of 2.0.0.
> > >>> > >> > > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > > I have been updating the PR following
> > >>> recent
> > >>> > >> > updates.
> > >>> > >> > > > > Just
> > >>> > >> > > > > > > > now, I
> > >>> > >> > > > > > > > > > > > rebased
> > >>> > >> > > > > > > > > > > > > > it against the latest trunk and
> updated
> > >>> the
> > >>> > zstd
> > >>> > >> > > > version
> > >>> > >> > > > > > into
> > >>> > >> > > > > > > > > > 1.3.5.
> > >>> > >> > > > > > > > > > > > If you
> > >>> > >> > > > > > > > > > > > > > need some request, don't hesitate to
> > >>> notify
> > >>> > me.
> > >>> > >> > (But
> > >>> > >> > > > not
> > >>> > >> > > > > > this
> > >>> > >> > > > > > > > > > thread
> > >>> > >> > > > > > > > > > > -
> > >>> > >> > > > > > > > > > > > just
> > >>> > >> > > > > > > > > > > > > > send me the message directly.)
> > >>> > >> > > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > > Best,
> > >>> > >> > > > > > > > > > > > > > Dongjin
> > >>> > >> > > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > > On Tue, Jul 10, 2018 at 11:57 PM Bobby
> > >>> Evans <
> > >>> > >> > > > > > > bo...@apache.org
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > >> I there any update on this.  The
> > >>> performance
> > >>> > >> > > > > improvements
> > >>> > >> > > > > > > are
> > >>> > >> > > > > > > > > > quite
> > >>> > >> > > > > > > > > > > > > >> impressive and I really would like to
> > >>> stop
> > >>> > >> forking
> > >>> > >> > > > kafka
> > >>> > >> > > > > > > just
> > >>> > >> > > > > > > > to
> > >>> > >> > > > > > > > > > get
> > >>> > >> > > > > > > > > > > > this
> > >>> > >> > > > > > > > > > > > > >> in.
> > >>> > >> > > > > > > > > > > > > >>
> > >>> > >> > > > > > > > > > > > > >> Thanks,
> > >>> > >> > > > > > > > > > > > > >>
> > >>> > >> > > > > > > > > > > > > >> Bobby
> > >>> > >> > > > > > > > > > > > > >>
> > >>> > >> > > > > > > > > > > > > >> On Wed, Jun 13, 2018 at 8:56 PM
> Dongjin
> > >>> Lee <
> > >>> > >> > > > > > > > dong...@apache.org
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > > > > >>
> > >>> > >> > > > > > > > > > > > > >> > Ismael,
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> > Oh, I forgot all of you are on
> > working
> > >>> > frenzy
> > >>> > >> > for
> > >>> > >> > > > 2.0!
> > >>> > >> > > > > > No
> > >>> > >> > > > > > > > > > problem,
> > >>> > >> > > > > > > > > > > > take
> > >>> > >> > > > > > > > > > > > > >> > your time. I am also working at
> > another
> > >>> > issue
> > >>> > >> > now.
> > >>> > >> > > > > Thank
> > >>> > >> > > > > > > you
> > >>> > >> > > > > > > > > for
> > >>> > >> > > > > > > > > > > > > >> letting me
> > >>> > >> > > > > > > > > > > > > >> > know.
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> > Best,
> > >>> > >> > > > > > > > > > > > > >> > Dongjin
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> > On Wed, Jun 13, 2018, 11:44 PM
> Ismael
> > >>> Juma
> > >>> > <
> > >>> > >> > > > > > > > isma...@gmail.com
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> > > Sorry for the delay Dongjin.
> > >>> Everyone is
> > >>> > >> busy
> > >>> > >> > > > > > finalising
> > >>> > >> > > > > > > > > > 2.0.0.
> > >>> > >> > > > > > > > > > > > This
> > >>> > >> > > > > > > > > > > > > >> KIP
> > >>> > >> > > > > > > > > > > > > >> > > seems like a great candidate for
> > >>> 2.1.0
> > >>> > and
> > >>> > >> > > > hopefully
> > >>> > >> > > > > > > there
> > >>> > >> > > > > > > > > > will
> > >>> > >> > > > > > > > > > > be
> > >>> > >> > > > > > > > > > > > > >> more
> > >>> > >> > > > > > > > > > > > > >> > of
> > >>> > >> > > > > > > > > > > > > >> > > a discussion next week. :)
> > >>> > >> > > > > > > > > > > > > >> > >
> > >>> > >> > > > > > > > > > > > > >> > > Ismael
> > >>> > >> > > > > > > > > > > > > >> > >
> > >>> > >> > > > > > > > > > > > > >> > > On Wed, 13 Jun 2018, 05:17
> Dongjin
> > >>> Lee, <
> > >>> > >> > > > > > > > dong...@apache.org
> > >>> > >> > > > > > > > > >
> > >>> > >> > > > > > > > > > > > wrote:
> > >>> > >> > > > > > > > > > > > > >> > >
> > >>> > >> > > > > > > > > > > > > >> > > > Hello. I just updated my draft
> > >>> > >> > implementation:
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > 1. Rebased to latest trunk
> > (commit
> > >>> > >> 5145d6b)
> > >>> > >> > > > > > > > > > > > > >> > > > 2. Apply ZStd 1.3.4
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > You can check out the
> > >>> implementation
> > >>> > from
> > >>> > >> > here
> > >>> > >> > > > > > > > > > > > > >> > > > <
> > >>> > >> https://github.com/apache/kafka/pull/2267
> > >>> > >> > >.
> > >>> > >> > > If
> > >>> > >> > > > > you
> > >>> > >> > > > > > > > > > > experience
> > >>> > >> > > > > > > > > > > > any
> > >>> > >> > > > > > > > > > > > > >> > > problem
> > >>> > >> > > > > > > > > > > > > >> > > > running it, don't hesitate to
> > give
> > >>> me a
> > >>> > >> > > mention.
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > Best,
> > >>> > >> > > > > > > > > > > > > >> > > > Dongjin
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > On Tue, Jun 12, 2018 at 6:50 PM
> > >>> Dongjin
> > >>> > >> Lee
> > >>> > >> > <
> > >>> > >> > > > > > > > > > > dong...@apache.org
> > >>> > >> > > > > > > > > > > > >
> > >>> > >> > > > > > > > > > > > > >> > wrote:
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > Here is the short conclusion
> > >>> about
> > >>> > the
> > >>> > >> > > license
> > >>> > >> > > > > > > > problem:
> > >>> > >> > > > > > > > > > *We
> > >>> > >> > > > > > > > > > > > can
> > >>> > >> > > > > > > > > > > > > >> use
> > >>> > >> > > > > > > > > > > > > >> > > zstd
> > >>> > >> > > > > > > > > > > > > >> > > > > and zstd-jni without any
> > >>> problem, but
> > >>> > >> we
> > >>> > >> > > need
> > >>> > >> > > > to
> > >>> > >> > > > > > > > include
> > >>> > >> > > > > > > > > > > their
> > >>> > >> > > > > > > > > > > > > >> > license,
> > >>> > >> > > > > > > > > > > > > >> > > > > e.g., BSD license.*
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > Both of BSD 2 Clause License
> &
> > 3
> > >>> > Clause
> > >>> > >> > > > License
> > >>> > >> > > > > > > > requires
> > >>> > >> > > > > > > > > > to
> > >>> > >> > > > > > > > > > > > > >> include
> > >>> > >> > > > > > > > > > > > > >> > the
> > >>> > >> > > > > > > > > > > > > >> > > > > license used, and BSD 3
> Clause
> > >>> > License
> > >>> > >> > > > requires
> > >>> > >> > > > > > that
> > >>> > >> > > > > > > > the
> > >>> > >> > > > > > > > > > > name
> > >>> > >> > > > > > > > > > > > of
> > >>> > >> > > > > > > > > > > > > >> the
> > >>> > >> > > > > > > > > > > > > >> > > > > contributor can't be used to
> > >>> endorse
> > >>> > or
> > >>> > >> > > > promote
> > >>> > >> > > > > > the
> > >>> > >> > > > > > > > > > product.
> > >>> > >> > > > > > > > > > > > > >> That's
> > >>> > >> > > > > > > > > > > > > >> > it
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > >
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >>
> > >>> > >> > http://www.mikestratton.net/2011/12/is-bsd-license-
> > >>> > >> > > > > > > > > > > > compatible-with-apache-2-0-license/
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > - They are not listed in the
> > >>> list of
> > >>> > >> > > > prohibited
> > >>> > >> > > > > > > > licenses
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> https://www.apache.org/legal/
> > >>> > >> > > > > > > > resolved.html#category-x>
> > >>> > >> > > > > > > > > > > also.
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > Here is how Spark did for it
> > >>> > >> > > > > > > > > > > > > >> > > > > <https://issues.apache.org/
> > >>> > >> > > > > > jira/browse/SPARK-19112
> > >>> > >> > > > > > > >:
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > - They made a directory
> > >>> dedicated to
> > >>> > >> the
> > >>> > >> > > > > > dependency
> > >>> > >> > > > > > > > > > license
> > >>> > >> > > > > > > > > > > > files
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> > >>> > >> > > > > > > https://github.com/apache/
> spark/tree/master/licenses
> > >>> > >> > > > > > > > >
> > >>> > >> > > > > > > > > > and
> > >>> > >> > > > > > > > > > > > added
> > >>> > >> > > > > > > > > > > > > >> > > > licenses
> > >>> > >> > > > > > > > > > > > > >> > > > > for Zstd
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > https://github.com/apache/spark/blob/master/licenses/
> > >>> > >> > > > > > > > > > > > LICENSE-zstd.txt
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > > > &
> > >>> > >> > > > > > > > > > > > > >> > > > > Zstd-jni
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> > >>> > >> > > > > > > > > > > > > >> > > >
> > >>> > >> > > > > > > > > > > > > >> > >
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> https://github.com/apache/
> > >>> > >> > > spark/blob/master/licenses/
> > >>> > >> > > > > > > > > > > > LICENSE-zstd-jni.txt
> > >>> > >> > > > > > > > > > > > > >> >
> > >>> > >> > > > > > > > > > > > > >> > > > > .
> > >>> > >> > > > > > > > > > > > > >> > > > > - Added a link to the
> original
> > >>> > license
> > >>> > >> > files
> > >>> > >> > > > in
> > >>> > >> > > > > > > > LICENSE.
> > >>> > >> > > > > > > > > > > > > >> > > > > <
> > >>> > >> > > > > https://github.com/apache/spark/pull/18805/files
> > >>> > >> > > > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > If needed, I can make a
> similar
> > >>> > update.
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > Thanks for pointing out this
> > >>> problem,
> > >>> > >> > > Viktor!
> > >>> > >> > > > > Nice
> > >>> > >> > > > > > > > > catch!
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > Best,
> > >>> > >> > > > > > > > > > > > > >> > > > > Dongjin
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > > On Mon, Jun 11, 2018 at 11:50
> > PM
> > >>> > >> Dongjin
> > >>> > >> > > Lee <
> > >>> > >> > > > > > > > > > > > dong...@apache.org>
> > >>> > >> > > > > > > > > > > > > >> > > wrote:
> > >>> > >> > > > > > > > > > > > > >> > > > >
> > >>> > >> > > > > > > > > > > > > >> > > > >> I greatly appreciate your
> > >>> > >> comprehensive
> > >>> > >> > > > > > reasoning.
> > >>> > >> > > > > > > > so:
> > >>> > >> > > > > > > > > +1
> > >>> > >> > > > > > > > > > > > for b
> > >>> > >> > > > > > > > > > > > > >> > until
> > >>> > >> > > > > > > > > > > > > >> > > > now.
> > >>> >
>

Reply via email to