-
Notifications
You must be signed in to change notification settings - Fork 14.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
KAFKA-7632: Support Compression Levels (KIP-390) #15516
Conversation
df9ca6e
to
2f54aac
Compare
d284cd3
to
71d84bd
Compare
@divijvaidya It seems you've done a bit of work around compression in the past. Can you take a look? Thanks |
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.
Thanks for the PR. Left some comments.
clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/Compression.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/Lz4BlockOutputStream.java
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/compress/GzipCompressionTest.java
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/compress/Lz4CompressionTest.java
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java
Outdated
Show resolved
Hide resolved
throw new UnsupportedCompressionTypeException("Produce requests to inter.broker.protocol.version < 2.1 broker " + | ||
"are not allowed to use ZStandard compression"); | ||
|
||
// No in place assignment situation 1 | ||
boolean inPlaceAssignment = sourceCompression == targetCompression; | ||
boolean inPlaceAssignment = sourceCompressionType == targetCompression.type(); |
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.
So we won't do re-compression if only level is different? I didn't see this in KIP. Maybe we should add it?
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.
The broker has no easy way of retrieving the level that the producer used when compressing the records. So if the compression codec matches, I decided to keep the compressed bytes instead of decompressing and compressing everything again as this would be wasteful, especially as the producer could have already used the same compression level.
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 agree. But I think we should mention this in the KIP-390 at least.
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.
Right, I updated the KIP.
public static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>gzip</code>."; | ||
public static final String COMPRESSION_LZ4_LEVEL_CONFIG = "compression.lz4.level"; | ||
public static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>lz4</code>."; | ||
public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = "compression.zstd.level"; | ||
public static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to <code>zstd</code>."; |
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.
nit: Should we provide the doc link for each compression type? It's hard to know which level means what.
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.
Do you mean a link to the compression library websites?
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 was thinking we added in the config description. Or maybe added in KIP-390 is good enough.
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.
@mimaison : Thanks for the PR. Made a pass of non-testing files. Left a few comments.
clients/src/main/java/org/apache/kafka/common/compress/GzipCompression.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/Compression.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/NoCompression.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/compress/NoCompression.java
Show resolved
Hide resolved
|
||
public static final int MIN_LEVEL = 1; | ||
public static final int MAX_LEVEL = 17; | ||
public static final int DEFAULT_LEVEL = 9; |
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.
So, every time we update the Lz4 library, we may need to update the above values? We probably want to add a note here.
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 hesitated defining these constants for this reason but these levels have not changed over 10 years [0], so hopefully this won't require a lot of maintenance.
clients/src/main/java/org/apache/kafka/common/compress/ZstdCompression.java
Outdated
Show resolved
Hide resolved
clients/src/main/java/org/apache/kafka/common/telemetry/internals/ClientTelemetryUtils.java
Outdated
Show resolved
Hide resolved
storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java
Outdated
Show resolved
Hide resolved
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.
@mimaison : Thanks for the updated PR. Made a pass of all files. Added a few more comments.
clients/src/test/java/org/apache/kafka/common/compress/GzipCompressionTest.java
Outdated
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/compress/GzipCompressionTest.java
Outdated
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
Outdated
Show resolved
Hide resolved
clients/src/test/java/org/apache/kafka/common/compress/NoCompressionTest.java
Outdated
Show resolved
Hide resolved
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.
@mimaison : Thanks for the updated PR. Just one more comment.
server-common/src/test/java/org/apache/kafka/server/record/BrokerCompressionTypeTest.java
Outdated
Show resolved
Hide resolved
I also added a couple of new tests in LogValidatorTest to check recompression only happens if the compression codec is different between the records from the producer and the topic configuration and does not happen if only the compression levels are different. |
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.
@mimaison: Thanks for the updated PR. Left one more comment.
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.
@mimaison : Thanks for the updated PR. LGTM assuming all the failed tests have been triaged.
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.
Had another look. LGTM! Just some comments to update the KIP. Thanks.
Thanks for the reviews! |
None of the test failures seem related, merging to trunk |
Reviewers: Jun Rao <[email protected]>, Luke Chen <[email protected]> Co-authored-by: Lee Dongjin <[email protected]>
Reviewers: Jun Rao <[email protected]>, Luke Chen <[email protected]> Co-authored-by: Lee Dongjin <[email protected]>
Reviewers: Jun Rao <[email protected]>, Luke Chen <[email protected]> Co-authored-by: Lee Dongjin <[email protected]>
Reviewers: Jun Rao <[email protected]>, Luke Chen <[email protected]> Co-authored-by: Lee Dongjin <[email protected]>
Has anybody noticed that the Linear Write test in KIP-390 is inaccurate?
|
Yeah it looks like the numbers are not accurate. To be honest it's a bit of a strange performance test. The Produce Test benchmark should be much more representative. However I did not reproduce that benchmark. It might be worth asking @dongjinleekr if he has any data points to share. |
Based on #10826 with updates to match the recent amends we made to KIP-390.
Committer Checklist (excluded from commit message)