Skip to content

KAFKA-7632: Support Compression Levels (KIP-390)#15516

Merged
mimaison merged 6 commits intoapache:trunkfrom
mimaison:kip-390
May 21, 2024
Merged

KAFKA-7632: Support Compression Levels (KIP-390)#15516
mimaison merged 6 commits intoapache:trunkfrom
mimaison:kip-390

Conversation

@mimaison
Copy link
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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.

Comment thread clients/src/main/java/org/apache/kafka/common/compress/Compression.java Outdated
Comment thread clients/src/test/java/org/apache/kafka/common/record/DefaultRecordBatchTest.java Outdated

// No in place assignment situation 1
boolean inPlaceAssignment = sourceCompression == targetCompression;
boolean inPlaceAssignment = sourceCompressionType == targetCompression.type();
Copy link
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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.

Comment thread clients/src/main/java/org/apache/kafka/common/compress/GzipCompression.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/compress/Compression.java Outdated
Comment thread clients/src/main/java/org/apache/kafka/common/compress/NoCompression.java Outdated

public static final int MIN_LEVEL = 1;
public static final int MAX_LEVEL = 17;
public static final int DEFAULT_LEVEL = 9;
Copy link
Copy Markdown
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
Copy Markdown
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

Comment thread clients/src/main/java/org/apache/kafka/common/compress/ZstdCompression.java Outdated
Comment thread storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java Outdated
Copy link
Copy Markdown
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.

Comment thread clients/src/test/java/org/apache/kafka/common/compress/GzipCompressionTest.java Outdated
Comment thread clients/src/test/java/org/apache/kafka/common/compress/GzipCompressionTest.java Outdated
Comment thread clients/src/test/java/org/apache/kafka/common/compress/NoCompressionTest.java Outdated
Comment thread core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala Outdated
Comment thread core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
Comment thread core/src/test/scala/unit/kafka/log/LogConfigTest.scala
Comment thread core/src/test/scala/unit/kafka/log/LogValidatorTest.scala Outdated
Comment thread core/src/main/scala/kafka/log/UnifiedLog.scala Outdated
Copy link
Copy Markdown
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
Copy Markdown
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
Copy Markdown
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.

Comment thread core/src/test/scala/unit/kafka/log/LogValidatorTest.scala Outdated
Copy link
Copy Markdown
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
Copy Markdown
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
Copy Markdown
Member Author

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

@mimaison
Copy link
Copy Markdown
Member Author

None of the test failures seem related, merging to trunk

@mimaison mimaison merged commit affe8da into apache:trunk May 21, 2024
@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
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
chiacyu pushed a commit to chiacyu/kafka that referenced this pull request Jun 1, 2024
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
TaiJuWu pushed a commit to TaiJuWu/kafka that referenced this pull request Jun 8, 2024
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
gongxuanzhang pushed a commit to gongxuanzhang/kafka that referenced this pull request Jun 12, 2024
Reviewers: Jun Rao <jun@confluent.io>,  Luke Chen <showuon@gmail.com>
Co-authored-by: Lee Dongjin <dongjin@apache.org>
@stanislavkozlovski
Copy link
Copy Markdown
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
Copy Markdown
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.

ByteBufferOutputStream bufferOutputStream,
final long deleteHorizonMs) {
byte magic = originalBatch.magic();
Compression compression = Compression.of(originalBatch.compressionType()).build();
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. I'm not sure whether there has been any discussion about the compression level used during compaction. The current implementation just applies the default level, but perhaps it should respect the topic's configured compression level

  2. Another brainstorm is to introduce a flag that allows compaction to use different compression. This would give users option to choose a different compression algorithm for older data.

@mimaison @junrao @showuon WDYT?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. Right, here we could use the level if specified. I expect most topics to use compression.type=producer but in case a specific compression type and level is set, that would make sense to use them.

  2. It's not something I thought about before. Do you think there are scenarios where the gains of picking a different level for older data would be significant enough to motivate such a feature?

Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That’s an interesting topic.
For 1., it would be intuitive and reasonable to use the compression level that is set. I’ll create a minor for it.

For 2., I’m not entirely sure, but one possible case is that users may prefer tighter compression for cold data to save space, especially if storage is cost-sensitive

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Right, here we could use the level if specified. I expect most topics to use compression.type=producer but in case a specific compression type and level is set, that would make sense to use them.

@Yunyung Could you please file a minor patch for it?

Do you think there are scenarios where the gains of picking a different level for older data would be significant enough to motivate such a feature?

The key point is the compression type rather than level. I received a request to compress old data during compaction. The change should be straightforward, so it seems acceptable.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@chia7712 : Previously, we considered using the topic level compression type instead of the one in the original batch. There is one subtle issue on batch size. During compaction, we group a set of segments so that the total size doesn't exceed 2GB. If we use a different compression type, the compacted data could be exceeding the max segment limit and failing the index append.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we use a different compression type, the compacted data could be
exceeding the max segment limit and failing the index append.

It appears that using a high compression level during ingestion can indeed trigger overflow issue during compaction. This is because the cleaner's size estimation becomes inaccurate when it rebuilds batches using the default compression level instead of the origin one

[2025-12-28 08:25:42,915] WARN [kafka-log-cleaner-thread-0]: Unexpected exception thrown when cleaning log Log(dir=/tmp/log-folder-0/chia-0, topicId=5vKt56jQQ_S3QRuqrNmrTw, topic=chia, partition=0, highWatermark=66028296, lastStableOffset=66028296, logStartOffset=0, logEndOffset=66028296). Marking its partition (chia-0) as uncleanable (org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread)
org.apache.kafka.storage.internals.log.LogCleaningException: Append of size 258080 bytes is too large for segment with current file position at 2147262463
    at org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.java:570)
    at org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.java:544)
    at org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.doWork(LogCleaner.java:513)
    at org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:136)
Caused by: java.lang.IllegalArgumentException: Append of size 258080 bytes is too large for segment with current file position at 2147262463
    at org.apache.kafka.common.record.FileRecords.append(FileRecords.java:196)
    at org.apache.kafka.storage.internals.log.LogSegment.append(LogSegment.java:260)
    at org.apache.kafka.storage.internals.log.Cleaner.cleanInto(Cleaner.java:405)
    at org.apache.kafka.storage.internals.log.Cleaner.cleanSegments(Cleaner.java:243)
    at org.apache.kafka.storage.internals.log.Cleaner.doClean(Cleaner.java:180)
    at org.apache.kafka.storage.internals.log.Cleaner.clean(Cleaner.java:127)
    at org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.java:596)
    at org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.java:565)
    ... 3 more

To mitigate this, I propose introducing a safety margin specifically for partitions that fail to compact due to overflows.

@mimaison @junrao @Yunyung WDYT?

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another option is to make the cleaning logic more generic so that it could produce more than 1 segment during each round of cleaning if the segment size limit is exceeded.

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Theoretically, having the cleaner produce multiple segments is the superior design. However, the current implementation strictly pre-groups the segments. Simply splitting an overflowing group would result in fragmented, undersized segments. To handle this gracefully would require major architectural surgery on the grouping logic.

Given that this overflow is a rare edge case, requiring a specific combination of large segments, significant compression drop, and low deletion rates, the adaptive safety margin is a much more pragmatic win.

@junrao wdyt ?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good find!
Thanks for following up

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.

6 participants