You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/08/08 10:41:23 UTC

[GitHub] [kafka] mimaison commented on a diff in pull request #10826: KAFKA-7632: Support Compression Level

mimaison commented on code in PR #10826:
URL: https://github.com/apache/kafka/pull/10826#discussion_r940033070


##########
clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java:
##########
@@ -169,6 +169,14 @@ public class TopicConfig {
         "accepts 'uncompressed' which is equivalent to no compression; and 'producer' which means retain the " +
         "original compression codec set by the producer.";
 
+    /**
+     * <code>compression.level</code>
+     */
+    public static final String COMPRESSION_LEVEL_CONFIG = "compression.level";
+    public static final String COMPRESSION_LEVEL_DOC = "The compression level for all data generated by the producer. The default level and valid value is up to "

Review Comment:
   What about:
   ```
   The compression level to use when compressing records broker side. This is only used if compression.type is set to gzip, lz4 or zstd. The default level and valid value is up to...
   ```



##########
clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java:
##########
@@ -289,8 +290,7 @@ private void testAppendLargeOldMessageFormat(CompressionType compressionType) th
     public void testLinger() throws Exception {
         int lingerMs = 10;
         RecordAccumulator accum = createTestRecordAccumulator(
-                1024 + DefaultRecordBatch.RECORD_BATCH_OVERHEAD, 10 * 1024, CompressionType.NONE, lingerMs);
-        accum.append(topic, partition1, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, false, time.milliseconds(), cluster);

Review Comment:
   Don't we need this line for the test to pass?



##########
clients/src/test/java/org/apache/kafka/common/compress/KafkaLZ4Test.java:
##########
@@ -113,9 +116,11 @@ public Stream<? extends Arguments> provideArguments(ExtensionContext context) {
             for (Payload payload : payloads)
                 for (boolean broken : Arrays.asList(false, true))
                     for (boolean ignore : Arrays.asList(false, true))
-                        for (boolean blockChecksum : Arrays.asList(false, true))
-                            for (boolean close : Arrays.asList(false, true))
-                                arguments.add(Arguments.of(new Args(broken, ignore, blockChecksum, close, payload)));
+                        // Available levels: 1, 2, ... 17.
+                        for (int compressionLevel = 1; compressionLevel < 18; ++compressionLevel)

Review Comment:
   Should we use `Lz4OutputStream.MIN_COMPRESSION_LEVEL` and `MAX_COMPRESSION_LEVEL` rather than hardcoding the levels?



##########
core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala:
##########
@@ -101,14 +108,21 @@ object TestLinearWriteSpeed {
     val messageSize = options.valueOf(messageSizeOpt).intValue
     val flushInterval = options.valueOf(flushIntervalOpt).longValue
     val compressionCodec = CompressionCodec.getCompressionCodec(options.valueOf(compressionCodecOpt))
+    val extraProps = CommandLineUtils.parseKeyValueArgs(options.valuesOf(propertyOpt).asScala)
     val rand = new Random
     rand.nextBytes(buffer.array)
     val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD)
     val createTime = System.currentTimeMillis
+    val properties = new Properties()
+    properties ++= extraProps
+    properties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000")

Review Comment:
   Ah right this is only created in order to call `getCompressionConfig`. For a second I was wondering where `localhost:9000` came from.



##########
clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java:
##########
@@ -523,6 +532,45 @@ boolean idempotenceEnabled() {
         return userConfiguredTransactions || idempotenceEnabled;
     }
 
+    public CompressionConfig getCompressionConfig(CompressionType compressionType) {
+        if (getString(ProducerConfig.COMPRESSION_LEVEL_CONFIG).isEmpty()) {

Review Comment:
   Thanks @dongjinleekr for the explanations. My suggested change was logically identical, it's just a bit shorter.



##########
clients/src/main/java/org/apache/kafka/common/compress/LZ4Config.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.compress;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.record.CompressionConfig;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.common.utils.ByteBufferOutputStream;
+
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.record.RecordBatch.MAGIC_VALUE_V0;
+
+public final class LZ4Config extends CompressionConfig {
+    private final int level;
+
+    private LZ4Config(int level) {
+        this.level = level;
+    }
+
+    @Override
+    public CompressionType type() {
+        return CompressionType.LZ4;
+    }
+
+    @Override
+    public OutputStream wrapForOutput(ByteBufferOutputStream buffer, byte messageVersion) {
+        try {
+            return new Lz4OutputStream(buffer, this.level, messageVersion == MAGIC_VALUE_V0);
+        } catch (Throwable e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    @Override
+    public InputStream wrapForInput(ByteBuffer inputBuffer, byte messageVersion, BufferSupplier decompressionBufferSupplier) {
+        try {
+            return new KafkaLZ4BlockInputStream(inputBuffer, decompressionBufferSupplier,
+                messageVersion == MAGIC_VALUE_V0);
+        } catch (Throwable e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    public static class Builder extends CompressionConfig.Builder<LZ4Config> {
+        private int level = Lz4OutputStream.DEFAULT_COMPRESSION_LEVEL;
+
+        public Builder level(int level) {
+            if (level < Lz4OutputStream.MIN_COMPRESSION_LEVEL || Lz4OutputStream.MAX_COMPRESSION_LEVEL < level) {
+                throw new IllegalArgumentException("lz4 doesn't support given compression level: " + level);

Review Comment:
   Can we include the min and max values in the error message?



##########
clients/src/main/java/org/apache/kafka/common/compress/ZstdConfig.java:
##########
@@ -70,4 +91,22 @@ public void release(ByteBuffer buffer) {
             throw new KafkaException(e);
         }
     }
+
+    public static class Builder extends CompressionConfig.Builder<ZstdConfig> {
+        private int level = DEFAULT_COMPRESSION_LEVEL;
+
+        public Builder level(int level) {
+            if (MAX_COMPRESSION_LEVEL < level) {

Review Comment:
   Should we also check the min level?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org