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

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from

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

@showuon showuon self-assigned this Apr 11, 2024
Co-authored-by: Lee Dongjin <dongjin@apache.org>
Copy link
Contributor

@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.

@@ -189,7 +189,7 @@ public RecordAccumulator(LogContext logContext,
BufferPool bufferPool) {
this(logContext,
batchSize,
compression,
compression,
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: wrong indent

Comment on lines +35 to +39
* Wrap bufferStream with an OutputStream that will compress data with this CompressionType.
* Note: Unlike {@link #wrapForInput}, this cannot take {@link ByteBuffer}s directly.
* Currently, MemoryRecordsBuilder writes to the underlying buffer in the given {@link ByteBufferOutputStream} after the compressed data has been written.
* In the event that the buffer needs to be expanded while writing the data, access to the underlying buffer needs to be preserved.
*/
Copy link
Contributor

Choose a reason for hiding this comment

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

Please add @param @return .

* @param buffer The {@link ByteBuffer} instance holding the data to decompress.
* @param messageVersion The record format version to use.
* @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported.
* For small record batches, allocating a potentially large buffer (64 KB for LZ4)
Copy link
Contributor

Choose a reason for hiding this comment

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

miss @return

Copy link
Contributor

Choose a reason for hiding this comment

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

indentation

Comment on lines +72 to +78
/*
* lz4-java provides two types of compressors; fastCompressor, which requires less memory but fast compression speed (with default compression level only),
* and highCompressor which requires more memory and slower speed but compresses more efficiently (with various compression level).
*
* For backward compatibility, Lz4BlockOutputStream uses fastCompressor with default compression level but, with the other level, it uses highCompressor.
*/
compressor = level == Lz4Compression.DEFAULT_LEVEL ? LZ4Factory.fastestInstance().fastCompressor() : LZ4Factory.fastestInstance().highCompressor(level);
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we add this comment in the lz4.level config description? I think users also need to know it.

Comment on lines -45 to -47
public static final int BLOCKSIZE_256KB = 5;
public static final int BLOCKSIZE_1MB = 6;
public static final int BLOCKSIZE_4MB = 7;
Copy link
Contributor

Choose a reason for hiding this comment

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

They can be removed because we have another KIP to address them. Fine.

Comment on lines +67 to +68
builder.level(GzipCompression.MIN_LEVEL);
builder.level(GzipCompression.MAX_LEVEL);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: assertDoesNotThrow

Comment on lines +116 to +117
builder.level(Lz4Compression.MIN_LEVEL);
builder.level(Lz4Compression.MAX_LEVEL);
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: assertDoesNotThrow

// Buffer containing compressed records to be used for creating zstd-jni stream
ByteBuffer recordsBuffer = compressedBuf.duplicate();
ByteBuffer recordsBuffer = spy(compressedBuf.duplicate());
Copy link
Contributor

Choose a reason for hiding this comment

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

Why should we spy this?

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
Contributor

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?

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
Contributor

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
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.

// Set input buffer (uncompressed) to 16 KB (none by default) and output buffer (compressed) to
// 8 KB (0.5 KB by default) to ensure reasonable performance in cases where the caller passes a small
// number of bytes to write (potentially a single byte)
return new BufferedOutputStream(new GzipOutputStream(buffer, 8 * 1024, this.level), 16 * 1024);
Copy link
Contributor

Choose a reason for hiding this comment

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

this.level => level Ditto in a few other classes.

* @param buffer The {@link ByteBuffer} instance holding the data to decompress.
* @param messageVersion The record format version to use.
* @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported.
* For small record batches, allocating a potentially large buffer (64 KB for LZ4)
Copy link
Contributor

Choose a reason for hiding this comment

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

indentation

CompressionType type();

/**
* Wrap bufferStream with an OutputStream that will compress data with this CompressionType.
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add the javadoc for messageVersion?


@Override
public int hashCode() {
return super.hashCode();
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm, we redefine equals() such that all objects of type NoCompression are equal, yet they have different hashcode?

}

public static class Builder implements Compression.Builder<NoCompression> {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we make the constructor of NoCompression private so that only the builder can be used? Ditto for other compression classes.


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.

}

// visible for testing
public ZstdInputStreamNoFinalizer wrapForZstdInput(ByteBuffer buffer, BufferSupplier decompressionBufferSupplier) throws IOException {
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we make this static?

ByteBufferOutputStream out = new ByteBufferOutputStream(512)) {

Copy link
Contributor

Choose a reason for hiding this comment

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

extra new line

@@ -388,6 +402,32 @@ public LogConfig(Map<?, ?> props, Set<String> overriddenConfigs) {
remoteLogConfig = new RemoteLogConfig(this);
}

private Optional<Compression> getCompression() {
BrokerCompressionType brokerCompressionType = BrokerCompressionType.forName(getString(TopicConfig.COMPRESSION_TYPE_CONFIG));
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we just use compressionType, which is previously set already?

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