mirror of https://github.com/apache/kafka.git
				
				
				
			MINOR: Change LogCompactionTester topics parameter to Set<String> (#20372)
issue: https://github.com/apache/kafka/pull/19905#discussion_r2282202312 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>
This commit is contained in:
		
							parent
							
								
									47bb46c10d
								
							
						
					
					
						commit
						511818e027
					
				|  | @ -43,8 +43,8 @@ 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.LinkedHashSet; | ||||
| import java.util.List; | ||||
| import java.util.Map; | ||||
| import java.util.Properties; | ||||
|  | @ -61,6 +61,8 @@ import joptsimple.OptionParser; | |||
| import joptsimple.OptionSet; | ||||
| import joptsimple.OptionSpec; | ||||
| 
 | ||||
| import static java.util.stream.Collectors.toCollection; | ||||
| 
 | ||||
| 
 | ||||
| /** | ||||
|  * This is a torture test that runs against an existing broker | ||||
|  | @ -246,9 +248,9 @@ public class LogCompactionTester { | |||
|         int sleepSecs = optionSet.valueOf(options.sleepSecsOpt); | ||||
| 
 | ||||
|         long testId = RANDOM.nextLong(); | ||||
|         String[] topics = IntStream.range(0, topicCount) | ||||
|         Set<String> topics = IntStream.range(0, topicCount) | ||||
|                 .mapToObj(i -> "log-cleaner-test-" + testId + "-" + i) | ||||
|                 .toArray(String[]::new); | ||||
|                 .collect(toCollection(LinkedHashSet::new)); | ||||
|         createTopics(brokerUrl, topics); | ||||
| 
 | ||||
|         System.out.println("Producing " + messages + " messages..to topics " + String.join(",", topics)); | ||||
|  | @ -278,7 +280,7 @@ public class LogCompactionTester { | |||
|     } | ||||
| 
 | ||||
| 
 | ||||
|     private static void createTopics(String brokerUrl, String[] topics) throws Exception { | ||||
|     private static void createTopics(String brokerUrl, Set<String> topics) throws Exception { | ||||
|         Properties adminConfig = new Properties(); | ||||
|         adminConfig.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, brokerUrl); | ||||
| 
 | ||||
|  | @ -286,7 +288,7 @@ public class LogCompactionTester { | |||
|             Map<String, String> topicConfigs = Map.of( | ||||
|                     TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT | ||||
|             ); | ||||
|             List<NewTopic> newTopics = Arrays.stream(topics) | ||||
|             List<NewTopic> newTopics = topics.stream() | ||||
|                     .map(name -> new NewTopic(name, 1, (short) 1).configs(topicConfigs)).toList(); | ||||
|             adminClient.createTopics(newTopics).all().get(); | ||||
| 
 | ||||
|  | @ -296,7 +298,7 @@ public class LogCompactionTester { | |||
|                     Set<String> allTopics = adminClient.listTopics().names().get(); | ||||
|                     pendingTopics.clear(); | ||||
|                     pendingTopics.addAll( | ||||
|                             Arrays.stream(topics) | ||||
|                             topics.stream() | ||||
|                                     .filter(topicName -> !allTopics.contains(topicName)) | ||||
|                                     .toList() | ||||
|                     ); | ||||
|  | @ -392,7 +394,7 @@ public class LogCompactionTester { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private static Path produceMessages(String brokerUrl, String[] topics, long messages, | ||||
|     private static Path produceMessages(String brokerUrl, Set<String> topics, long messages, | ||||
|                                         String compressionType, int dups, int percentDeletes) throws IOException { | ||||
|         Map<String, Object> producerProps = Map.of( | ||||
|                 ProducerConfig.MAX_BLOCK_MS_CONFIG, String.valueOf(Long.MAX_VALUE), | ||||
|  | @ -408,8 +410,10 @@ public class LogCompactionTester { | |||
| 
 | ||||
|             try (BufferedWriter producedWriter = Files.newBufferedWriter( | ||||
|                     producedFilePath, StandardCharsets.UTF_8)) { | ||||
|                 for (long i = 0; i < messages * topics.length; i++) { | ||||
|                     String topic = topics[(int) (i % topics.length)]; | ||||
|                 List<String> topicsList = List.copyOf(topics); | ||||
|                 int size = topicsList.size(); | ||||
|                 for (long i = 0; i < messages * size; i++) { | ||||
|                     String topic = topicsList.get((int) (i % size)); | ||||
|                     int key = RANDOM.nextInt(keyCount); | ||||
|                     boolean delete = (i % 100) < percentDeletes; | ||||
|                     ProducerRecord<byte[], byte[]> record; | ||||
|  | @ -430,14 +434,14 @@ public class LogCompactionTester { | |||
|         } | ||||
|     } | ||||
| 
 | ||||
|     private static Path consumeMessages(String brokerUrl, String[] topics) throws IOException { | ||||
|     private static Path consumeMessages(String brokerUrl, Set<String> topics) throws IOException { | ||||
| 
 | ||||
|         Path consumedFilePath = Files.createTempFile("kafka-log-cleaner-consumed-", ".txt"); | ||||
|         System.out.println("Logging consumed messages to " + consumedFilePath); | ||||
| 
 | ||||
|         try (Consumer<String, String> consumer = createConsumer(brokerUrl); | ||||
|              BufferedWriter consumedWriter = Files.newBufferedWriter(consumedFilePath, StandardCharsets.UTF_8)) { | ||||
|             consumer.subscribe(Arrays.asList(topics)); | ||||
|             consumer.subscribe(topics); | ||||
|             while (true) { | ||||
|                 ConsumerRecords<String, String> consumerRecords = consumer.poll(Duration.ofSeconds(20)); | ||||
|                 if (consumerRecords.isEmpty()) return consumedFilePath; | ||||
|  |  | |||
		Loading…
	
		Reference in New Issue