Skip to content
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

Merged
merged 6 commits into from
May 21, 2024

Conversation

mimaison
Copy link
Member

@mimaison mimaison commented Mar 11, 2024

Based on #10826 with updates to match the recent amends we made to KIP-390.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@mimaison mimaison force-pushed the kip-390 branch 3 times, most recently from df9ca6e to 2f54aac Compare March 25, 2024 08:47
@mimaison mimaison force-pushed the kip-390 branch 3 times, most recently from d284cd3 to 71d84bd Compare April 10, 2024 09:04
@mimaison mimaison marked this pull request as ready for review April 10, 2024 10:00
@mimaison
Copy link
Member Author

@divijvaidya It seems you've done a bit of work around compression in the past. Can you take a look? Thanks

Copy link
Member

@showuon showuon left a 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.

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();
Copy link
Member

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?

Copy link
Member Author

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.

Copy link
Member

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.

Copy link
Member Author

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.

Comment on lines +178 to +182
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>.";
Copy link
Member

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.

Copy link
Member Author

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?

Copy link
Member

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.

Copy link
Contributor

@junrao junrao left a 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.


public static final int MIN_LEVEL = 1;
public static final int MAX_LEVEL = 17;
public static final int DEFAULT_LEVEL = 9;
Copy link
Contributor

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.

Copy link
Member Author

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.

0: https://github.com/lz4/lz4-java/blame/master/src/java/net/jpountz/lz4/LZ4Constants.java#L23-L24

Copy link
Contributor

@junrao junrao left a 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.

Copy link
Contributor

@junrao junrao left a 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.

@mimaison
Copy link
Member Author

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.

Copy link
Contributor

@junrao junrao left a 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.

Copy link
Contributor

@junrao junrao left a 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.

Copy link
Member

@showuon showuon left a 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.

@mimaison
Copy link
Member Author

Thanks for the reviews!
I had to rebase again so I'll wait for the CI to complete.

@mimaison
Copy link
Member Author

None of the test failures seem related, merging to trunk

@mimaison mimaison merged commit affe8da into apache:trunk May 21, 2024
1 check failed
@mimaison mimaison deleted the kip-390 branch May 21, 2024 15:58
rreddy-22 pushed a commit to rreddy-22/kafka-rreddy that referenced this pull request May 24, 2024
chiacyu pushed a commit to chiacyu/kafka that referenced this pull request Jun 1, 2024
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Jun 8, 2024
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
@stanislavkozlovski
Copy link
Contributor

Has anybody noticed that the Linear Write test in KIP-390 is inaccurate?

  1. It suggests that the write speed on a broker is 22GB/s. I wasn't able to find an SSD on the market in 2024 (6 years later) that supports this throughput
  2. The reason for this, I think, is that it inaccurately sets the benchmark test to write just 8192 bytes --bytes 8192 and the size to write is 8192 itself, so the test seems to just be comparing the nanoseconds it takes for the very first write and then extrapolating further to a full second.

image

@mimaison
Copy link
Member Author

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.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

4 participants