2727import org .apache .kafka .clients .producer .ProducerConfig ;
2828import org .apache .kafka .clients .producer .ProducerRecord ;
2929import org .apache .kafka .common .config .TopicConfig ;
30+ import org .apache .kafka .common .record .CompressionType ;
3031import org .apache .kafka .common .serialization .ByteArraySerializer ;
3132import org .apache .kafka .common .serialization .StringDeserializer ;
3233import org .apache .kafka .common .utils .Exit ;
4344import java .nio .file .Path ;
4445import java .time .Duration ;
4546import java .util .ArrayList ;
47+ import java .util .HashMap ;
4648import java .util .Iterator ;
4749import java .util .LinkedHashSet ;
4850import java .util .List ;
@@ -86,6 +88,7 @@ public class LogCompactionTester {
8688 public static class Options {
8789 public final OptionSpec <Long > numMessagesOpt ;
8890 public final OptionSpec <String > messageCompressionOpt ;
91+ public final OptionSpec <Integer > compressionLevelOpt ;
8992 public final OptionSpec <Integer > numDupsOpt ;
9093 public final OptionSpec <String > brokerOpt ;
9194 public final OptionSpec <Integer > topicsOpt ;
@@ -108,6 +111,12 @@ public Options(OptionParser parser) {
108111 .ofType (String .class )
109112 .defaultsTo ("none" );
110113
114+ compressionLevelOpt = parser
115+ .accepts ("compression-level" , "The compression level to use with the specified compression type." )
116+ .withOptionalArg ()
117+ .describedAs ("level" )
118+ .ofType (Integer .class );
119+
111120 numDupsOpt = parser
112121 .accepts ("duplicates" , "The number of duplicates for each key." )
113122 .withRequiredArg ()
@@ -240,7 +249,8 @@ public static void main(String[] args) throws Exception {
240249 CommandLineUtils .checkRequiredArgs (parser , optionSet , options .brokerOpt , options .numMessagesOpt );
241250
242251 long messages = optionSet .valueOf (options .numMessagesOpt );
243- String compressionType = optionSet .valueOf (options .messageCompressionOpt );
252+ CompressionType compressionType = CompressionType .forName (optionSet .valueOf (options .messageCompressionOpt ));
253+ Integer compressionLevel = optionSet .valueOf (options .compressionLevelOpt );
244254 int percentDeletes = optionSet .valueOf (options .percentDeletesOpt );
245255 int dups = optionSet .valueOf (options .numDupsOpt );
246256 String brokerUrl = optionSet .valueOf (options .brokerOpt );
@@ -256,7 +266,8 @@ public static void main(String[] args) throws Exception {
256266 System .out .println ("Producing " + messages + " messages..to topics " + String .join ("," , topics ));
257267 Path producedDataFilePath = produceMessages (
258268 brokerUrl , topics , messages ,
259- compressionType , dups , percentDeletes );
269+ compressionType , compressionLevel ,
270+ dups , percentDeletes );
260271 System .out .println ("Sleeping for " + sleepSecs + "seconds..." );
261272 TimeUnit .MILLISECONDS .sleep (sleepSecs * 1000L );
262273 System .out .println ("Consuming messages..." );
@@ -395,12 +406,22 @@ private static void require(boolean requirement, String message) {
395406 }
396407
397408 private static Path produceMessages (String brokerUrl , Set <String > topics , long messages ,
398- String compressionType , int dups , int percentDeletes ) throws IOException {
399- Map <String , Object > producerProps = Map .of (
400- ProducerConfig .MAX_BLOCK_MS_CONFIG , String .valueOf (Long .MAX_VALUE ),
401- ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , brokerUrl ,
402- ProducerConfig .COMPRESSION_TYPE_CONFIG , compressionType
403- );
409+ CompressionType compressionType , Integer compressionLevel ,
410+ int dups , int percentDeletes ) throws IOException {
411+ Map <String , Object > producerProps = new HashMap <>();
412+ producerProps .put (ProducerConfig .MAX_BLOCK_MS_CONFIG , String .valueOf (Long .MAX_VALUE ));
413+ producerProps .put (ProducerConfig .BOOTSTRAP_SERVERS_CONFIG , brokerUrl );
414+ producerProps .put (ProducerConfig .COMPRESSION_TYPE_CONFIG , compressionType .name );
415+
416+ if (compressionLevel != null ) {
417+ switch (compressionType ) {
418+ case GZIP -> producerProps .put (ProducerConfig .COMPRESSION_GZIP_LEVEL_CONFIG , compressionLevel );
419+ case LZ4 -> producerProps .put (ProducerConfig .COMPRESSION_LZ4_LEVEL_CONFIG , compressionLevel );
420+ case ZSTD -> producerProps .put (ProducerConfig .COMPRESSION_ZSTD_LEVEL_CONFIG , compressionLevel );
421+ default -> System .out .println ("Warning: Compression level " + compressionLevel + " is ignored for compression type "
422+ + compressionType .name + ". Only gzip, lz4, and zstd support compression levels." );
423+ }
424+ }
404425
405426 try (KafkaProducer <byte [], byte []> producer = new KafkaProducer <>(
406427 producerProps , new ByteArraySerializer (), new ByteArraySerializer ())) {
0 commit comments