-
Notifications
You must be signed in to change notification settings - Fork 3.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add support for KIP-110 ZSTD compression #2053
Conversation
305e898
to
7efbdfe
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Wow, that was quick, the AK PR was merged two days ago!
This looks great, just needs a couple of minor cosmetic changes.
src/rdzstd.c
Outdated
case ZSTD_CONTENTSIZE_UNKNOWN: | ||
/* Decompressed size cannot be determined, make a guess */ | ||
out_bufsize = inlen * 2; | ||
case ZSTD_CONTENTSIZE_ERROR: |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should be a break; here, right?
src/rdzstd.c
Outdated
/* Check if the destination size is too small */ | ||
ZSTD_ErrorCode err = ZSTD_getErrorCode(ret); | ||
if (err == ZSTD_error_dstSize_tooSmall) { | ||
out_bufsize *= 2; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I can't find the source now, and I don't really remember why, but it is supposedly better to multiply by 1.75 than 2 when increasing buffers.
https:/edenhill/librdkafka/blob/master/src/rdkafka_lz4.c#L268
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Add this: rd_atomic64_add(&rkb->rkb_c.zbuf_grow, 1);
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Updated
src/rdzstd.c
Outdated
ZSTD_ErrorCode err = ZSTD_getErrorCode(ret); | ||
if (err == ZSTD_error_dstSize_tooSmall) { | ||
out_bufsize *= 2; | ||
char *newp = rd_realloc(decompressed, out_bufsize); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
variable definitions need to go at the top of the scope to be compatible with older non C99 compilers (MSVC).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I updated it but it doesn't compile with c89 (no inline etc.), is this is just for MSVC compatibility?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, at least that's the most common weird old compiler we come across frequently, and it is stuck somewhere between c89 and c99 :)
src/rdzstd.c
Outdated
} | ||
|
||
/* Check if the destination size is too small */ | ||
ZSTD_ErrorCode err = ZSTD_getErrorCode(ret); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
variable definititions need to get at the top of the scope.
err
shadows the top-level rd_kafka_resp_err_t err
, better call this one zerr
src/rdzstd.c
Outdated
} | ||
} | ||
|
||
rd_assert(rd_slice_remains(slice) == 0); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Better to error out here instead of crashing since the input data is from an external source.
src/rdzstd.h
Outdated
@@ -0,0 +1,52 @@ | |||
/* | |||
* Copyright (c) 2018, Marek Vavrusa <[email protected] |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
dito, rename to rdkafka_zstd.h, add copyright, add trailing >
CONFIGURATION.md
Outdated
@@ -140,3 +140,4 @@ offset.store.method | C | file, broker | broker | |||
consume.callback.max.messages | C | 0 .. 1000000 | 0 | Maximum number of messages to dispatch in one `rd_kafka_consume_callback*()` call (0 = unlimited) <br>*Type: integer* | |||
|
|||
### C/P legend: C = Consumer, P = Producer, * = both | |||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This file is automatically generated but seems to be missing the zstd stuff; make sure you build the source with make
at least once to let auto-generation kick in.
LICENSE.zstd
Outdated
@@ -0,0 +1,30 @@ | |||
BSD License |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we're not including the zstd source we shouldn't include this license.
LICENSES.txt
Outdated
@@ -342,3 +342,37 @@ For the files wingetopt.c wingetopt.h downloaded from https:/alex85k | |||
*/ | |||
|
|||
|
|||
LICENSE.zstd |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
dito
This will go in v1.0.0, which is currently the |
Yeah, we've been using zstd in Kafka for a while at Cloudflare, but it only currently "works" with Sarama, so I can't move without it. |
Move to what? :) v1.0.0 is scheduled for release within a month. |
I wrote the Kafka table engine for ClickHouse using librdkafka, but can't use it with most topics without zstd. The librdkafka is embedded as a submodule there, so I don't have to wait for the release tag before testing it. |
Rebased to |
Thanks for reviewing! 🙌 |
@vavrusa did you get it to work in your case? We upgraded to confluent kafka 5.1.0 / 2.1.0 and set the broker default compression to ZSTD. %7|1546470506.414|APIVERSION|rdkafka#consumer-1| [thrd:sasl_plaintext://xxx/bootstrap]: sasl_plaintext://xxx/bootstrap: Feature ZSTD: Produce (7..7) supported by broker 76 is the error introduced with KIP-110: UNSUPPORTED_COMPRESSION_TYPE. Maybe we need to bump the v4 fetch version somehow as per "Zstd will only be allowed for the bumped fetch API ".? or am i missing something? version: |
Yes, it's been working for some time. Maybe the negotiation isn't working properly for your case? I don't know that much about Kafka itself to help you, sorry! |
thanks for confirming! |
seems fetch api v10 support was missing, by adding the the fetch protocol support for v10 i got it to work. i can clean up and submit a PR. |
This was introduced recently in https://cwiki.apache.org/confluence/display/KAFKA/KIP-110%3A+Add+Codec+for+ZStandard+Compression
And merged in apache/kafka#2267
I'm going to need some help figuring out how to test this. I think I added the prerequisites right - Fetch APIv10 and Produce APIv7.