Skip to content

Commit 511818e

Browse files
authored
MINOR: Change LogCompactionTester topics parameter to Set<String> (#20372)
issue: #19905 (comment) What: Change `String[] topics` to `Set<String> topics` throughout `LogCompactionTester`. Why: `Set<String>` is more modern and reduces the need for array-to-collection conversions. Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi <kitingiao@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Lan Ding <isDing_L@163.com>, Kuan-Po Tseng <brandboat@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
1 parent 47bb46c commit 511818e

File tree

1 file changed

+15
-11
lines changed

1 file changed

+15
-11
lines changed

tools/src/main/java/org/apache/kafka/tools/LogCompactionTester.java

Lines changed: 15 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -43,8 +43,8 @@
4343
import java.nio.file.Path;
4444
import java.time.Duration;
4545
import java.util.ArrayList;
46-
import java.util.Arrays;
4746
import java.util.Iterator;
47+
import java.util.LinkedHashSet;
4848
import java.util.List;
4949
import java.util.Map;
5050
import java.util.Properties;
@@ -61,6 +61,8 @@
6161
import joptsimple.OptionSet;
6262
import joptsimple.OptionSpec;
6363

64+
import static java.util.stream.Collectors.toCollection;
65+
6466

6567
/**
6668
* This is a torture test that runs against an existing broker
@@ -246,9 +248,9 @@ public static void main(String[] args) throws Exception {
246248
int sleepSecs = optionSet.valueOf(options.sleepSecsOpt);
247249

248250
long testId = RANDOM.nextLong();
249-
String[] topics = IntStream.range(0, topicCount)
251+
Set<String> topics = IntStream.range(0, topicCount)
250252
.mapToObj(i -> "log-cleaner-test-" + testId + "-" + i)
251-
.toArray(String[]::new);
253+
.collect(toCollection(LinkedHashSet::new));
252254
createTopics(brokerUrl, topics);
253255

254256
System.out.println("Producing " + messages + " messages..to topics " + String.join(",", topics));
@@ -278,15 +280,15 @@ public static void main(String[] args) throws Exception {
278280
}
279281

280282

281-
private static void createTopics(String brokerUrl, String[] topics) throws Exception {
283+
private static void createTopics(String brokerUrl, Set<String> topics) throws Exception {
282284
Properties adminConfig = new Properties();
283285
adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl);
284286

285287
try (Admin adminClient = Admin.create(adminConfig)) {
286288
Map<String, String> topicConfigs = Map.of(
287289
TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT
288290
);
289-
List<NewTopic> newTopics = Arrays.stream(topics)
291+
List<NewTopic> newTopics = topics.stream()
290292
.map(name -> new NewTopic(name, 1, (short) 1).configs(topicConfigs)).toList();
291293
adminClient.createTopics(newTopics).all().get();
292294

@@ -296,7 +298,7 @@ private static void createTopics(String brokerUrl, String[] topics) throws Excep
296298
Set<String> allTopics = adminClient.listTopics().names().get();
297299
pendingTopics.clear();
298300
pendingTopics.addAll(
299-
Arrays.stream(topics)
301+
topics.stream()
300302
.filter(topicName -> !allTopics.contains(topicName))
301303
.toList()
302304
);
@@ -392,7 +394,7 @@ private static void require(boolean requirement, String message) {
392394
}
393395
}
394396

395-
private static Path produceMessages(String brokerUrl, String[] topics, long messages,
397+
private static Path produceMessages(String brokerUrl, Set<String> topics, long messages,
396398
String compressionType, int dups, int percentDeletes) throws IOException {
397399
Map<String, Object> producerProps = Map.of(
398400
ProducerConfig.MAX_BLOCK_MS_CONFIG, String.valueOf(Long.MAX_VALUE),
@@ -408,8 +410,10 @@ producerProps, new ByteArraySerializer(), new ByteArraySerializer())) {
408410

409411
try (BufferedWriter producedWriter = Files.newBufferedWriter(
410412
producedFilePath, StandardCharsets.UTF_8)) {
411-
for (long i = 0; i < messages * topics.length; i++) {
412-
String topic = topics[(int) (i % topics.length)];
413+
List<String> topicsList = List.copyOf(topics);
414+
int size = topicsList.size();
415+
for (long i = 0; i < messages * size; i++) {
416+
String topic = topicsList.get((int) (i % size));
413417
int key = RANDOM.nextInt(keyCount);
414418
boolean delete = (i % 100) < percentDeletes;
415419
ProducerRecord<byte[], byte[]> record;
@@ -430,14 +434,14 @@ record = new ProducerRecord<>(topic,
430434
}
431435
}
432436

433-
private static Path consumeMessages(String brokerUrl, String[] topics) throws IOException {
437+
private static Path consumeMessages(String brokerUrl, Set<String> topics) throws IOException {
434438

435439
Path consumedFilePath = Files.createTempFile("kafka-log-cleaner-consumed-", ".txt");
436440
System.out.println("Logging consumed messages to " + consumedFilePath);
437441

438442
try (Consumer<String, String> consumer = createConsumer(brokerUrl);
439443
BufferedWriter consumedWriter = Files.newBufferedWriter(consumedFilePath, StandardCharsets.UTF_8)) {
440-
consumer.subscribe(Arrays.asList(topics));
444+
consumer.subscribe(topics);
441445
while (true) {
442446
ConsumerRecords<String, String> consumerRecords = consumer.poll(Duration.ofSeconds(20));
443447
if (consumerRecords.isEmpty()) return consumedFilePath;

0 commit comments

Comments
 (0)