yunchipang commented on code in PR #19905: URL: https://github.com/apache/kafka/pull/19905#discussion_r2165992040
########## tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java: ########## @@ -0,0 +1,344 @@ +/* + * 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.test.TestUtils; +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.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.concurrent.CompletableFuture; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import joptsimple.OptionParser; +import joptsimple.OptionSet; + + +/** + * 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 { + private static final Random RANDOM = new Random(); + + public static void main(String[] args) throws Exception { + + OptionParser parser = new OptionParser(false); + LogCompactionTesterOptions options = new LogCompactionTesterOptions(parser); + + OptionSet optionSet = parser.parse(args); + if (args.length == 0) { + CommandLineUtils.printUsageAndExit(parser, + "A tool to test log compaction. Valid options are: "); + } + + CommandLineUtils.checkRequiredArgs(parser, optionSet, options.brokerOpt, options.numMessagesOpt); + + long messages = optionSet.valueOf(options.numMessagesOpt); + String compressionType = optionSet.valueOf(options.messageCompressionOpt); + int percentDeletes = optionSet.valueOf(options.percentDeletesOpt); + int dups = optionSet.valueOf(options.numDupsOpt); + String brokerUrl = optionSet.valueOf(options.brokerOpt); + int topicCount = optionSet.valueOf(options.topicsOpt); + int sleepSecs = optionSet.valueOf(options.sleepSecsOpt); + + long testId = RANDOM.nextLong(); + String[] topics = IntStream.range(0, topicCount) + .mapToObj(i -> "log-cleaner-test-" + testId + "-" + i) + .toArray(String[]::new); + createTopics(brokerUrl, topics); + + System.out.println("Producing " + messages + " messages..to topics " + String.join(",", topics)); + Path producedDataFilePath = produceMessages( + brokerUrl, topics, messages, + compressionType, dups, percentDeletes); + System.out.println("Sleeping for " + sleepSecs + "seconds..."); + Thread.sleep(sleepSecs * 1000L); + System.out.println("Consuming messages..."); + Path consumedDataFilePath = consumeMessages(brokerUrl, topics); + + long producedLines = lineCount(producedDataFilePath); + long consumedLines = lineCount(consumedDataFilePath); + double reduction = 100 * (1.0 - (double) consumedLines / producedLines); + + System.out.printf( + "%d rows of data produced, %d rows of data consumed (%.1f%% reduction).%n", + producedLines, consumedLines, reduction); + + System.out.println("De-duplicating and validating output files..."); + validateOutput(producedDataFilePath.toFile(), consumedDataFilePath.toFile()); + + Files.deleteIfExists(producedDataFilePath); + Files.deleteIfExists(consumedDataFilePath); + // if you change this line, we need to update test_log_compaction_tool.py system test + System.out.println("Data verification is completed"); + } + + + private static void createTopics(String brokerUrl, String[] topics) throws Exception { + Properties adminConfig = new Properties(); + adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl); + + try (Admin adminClient = Admin.create(adminConfig)) { + Map<String, String> topicConfigs = Map.of( + TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT + ); + List<NewTopic> newTopics = Arrays.stream(topics) + .map(name -> new NewTopic(name, 1, (short) 1).configs(topicConfigs)).toList(); + adminClient.createTopics(newTopics).all().get(); + + final List<String> pendingTopics = new ArrayList<>(); + TestUtils.waitForCondition(() -> { + try { + Set<String> allTopics = adminClient.listTopics().names().get(); Review Comment: @chia7712 @TaiJuWu for the current CI error: ``` * What went wrong: Execution failed for task ':tools:compileJava'. > Compilation failed; see the compiler output below. /home/runner/work/kafka/kafka/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java:32: error: TestUtils is not public in org.apache.kafka.common.test; cannot be accessed from outside package import org.apache.kafka.common.test.TestUtils; ^ /home/runner/work/kafka/kafka/tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java:148: error: TestUtils is not public in org.apache.kafka.common.test; cannot be accessed from outside package TestUtils.waitForCondition(() -> { ^ 2 errors * Try: > Check your code and dependencies to fix the compilation error(s) ``` I tried importing `org.apache.kafka.test.TestUtils`, but it’s not accessible from production code (and I assume we shouldn’t make it public). Is there a recommended approach—e.g., is there a production-accessible equivalent, or should I just implement a local helper? -- 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