Yunyung commented on code in PR #19905: URL: https://github.com/apache/kafka/pull/19905#discussion_r2198269571
########## tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java: ########## @@ -0,0 +1,529 @@ +/* + * 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.tools; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Exit; +import org.apache.kafka.server.util.CommandLineUtils; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.File; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Random; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import joptsimple.OptionParser; +import joptsimple.OptionSet; +import joptsimple.OptionSpec; + + +/** + * This is a torture test that runs against an existing broker + * <p> + * Here is how it works: + * <p> + * It produces a series of specially formatted messages to one or more partitions. Each message it produces + * it logs out to a text file. The messages have a limited set of keys, so there is duplication in the key space. + * <p> + * The broker will clean its log as the test runs. + * <p> + * When the specified number of messages have been produced we create a consumer and consume all the messages in the topic + * and write that out to another text file. + * <p> + * Using a stable unix sort we sort both the producer log of what was sent and the consumer log of what was retrieved by the message key. + * Then we compare the final message in both logs for each key. If this final message is not the same for all keys we + * print an error and exit with exit code 1, otherwise we print the size reduction and exit with exit code 0. + */ +public class LogCompactionTester { + + public static class Options { Review Comment: This can be rewritten using `CommandDefaultOptions`. For example:example:https://github.com/apache/kafka/blob/27383970b629775bf6368bf10e58122b3b709522/tools/src/main/java/org/apache/kafka/tools/ConsumerPerformance.java#L247 But I’m fine with this now. -- 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-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org