MINOR: Cleanup JMH-Benchmarks Module (#19791)

Now that Kafka supports Java 17, this PR makes some changes in
jmh-benchmarks module. The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Reviewers: Jhen-Yung Hsu <jhenyunghsu@gmail.com>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Sanskar Jhajharia 2025-07-02 18:23:57 +05:30 committed by GitHub
parent 28c53ba09a
commit 220ff4f774
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
21 changed files with 56 additions and 78 deletions

View File

@ -54,7 +54,6 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.IOException;
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -100,7 +99,7 @@ public class AuthorizerBenchmark {
// most map entries. In such cases, we rely on the filtering based on `String.startsWith`
// to return the matching ACLs. Using a more efficient data structure (e.g. a prefix
// tree) should improve performance significantly.
actions = Collections.singletonList(new Action(AclOperation.WRITE,
actions = List.of(new Action(AclOperation.WRITE,
new ResourcePattern(ResourceType.TOPIC, resourceNamePrefix + 95, PatternType.LITERAL),
1, true, true));
authorizeContext = new RequestContext(new RequestHeader(ApiKeys.PRODUCE, Integer.valueOf(1).shortValue(),

View File

@ -47,7 +47,6 @@ import java.util.List;
import java.util.Random;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.apache.kafka.common.acl.AclOperation.READ;
@ -98,7 +97,7 @@ public class StandardAuthorizerUpdateBenchmark {
return aclsForResource(resourcePattern);
})
.flatMap(Collection::stream)
.collect(Collectors.toList());
.toList();
}
private List<StandardAclWithId> aclsForResource(ResourcePattern pattern) {
@ -109,7 +108,7 @@ public class StandardAuthorizerUpdateBenchmark {
return new StandardAcl(pattern.resourceType(), pattern.name(), pattern.patternType(), p, h, READ, ALLOW);
})
.map(this::withId)
.collect(Collectors.toList());
.toList();
}
private StandardAclWithId withId(StandardAcl acl) {

View File

@ -35,8 +35,6 @@ import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.MetadataProvenance;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -145,7 +143,7 @@ public class AssignorBenchmarkUtils {
return new GroupSpecImpl(
memberSpecs,
subscriptionType,
Collections.emptyMap()
Map.of()
);
}
@ -335,7 +333,7 @@ public class AssignorBenchmarkUtils {
delta.replay(new PartitionRecord()
.setTopicId(topicId)
.setPartitionId(i)
.setReplicas(Arrays.asList(i % 4, (i + 1) % 4)));
.setReplicas(List.of(i % 4, (i + 1) % 4)));
}
}
}

View File

@ -40,11 +40,11 @@ import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.DEFAULT_GENERATION;
@ -161,7 +161,7 @@ public class ClientSideAssignorBenchmark {
partitions.addAll(partitionInfos(topicName, partitionsPerTopicCount, nodes));
}
metadata = new Cluster("test-cluster", nodes, partitions, Collections.emptySet(), Collections.emptySet());
metadata = new Cluster("test-cluster", nodes, partitions, Set.of(), Set.of());
}
private void addMemberSubscriptions() {
@ -230,7 +230,7 @@ public class ClientSideAssignorBenchmark {
return new ConsumerPartitionAssignor.Subscription(
topics,
null,
Collections.emptyList(),
List.of(),
DEFAULT_GENERATION,
rackId
);

View File

@ -48,7 +48,6 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -121,7 +120,7 @@ public class ServerSideAssignorBenchmark {
private GroupSpec groupSpec;
private List<String> allTopicNames = Collections.emptyList();
private List<String> allTopicNames = List.of();
private MetadataImage metadataImage = MetadataImage.EMPTY;
@ -196,14 +195,14 @@ public class ServerSideAssignorBenchmark {
for (String memberId : groupSpec.memberIds()) {
MemberAssignment memberAssignment = members.getOrDefault(
memberId,
new MemberAssignmentImpl(Collections.emptyMap())
new MemberAssignmentImpl(Map.of())
);
updatedMemberSpec.put(memberId, new MemberSubscriptionAndAssignmentImpl(
groupSpec.memberSubscription(memberId).rackId(),
Optional.empty(),
groupSpec.memberSubscription(memberId).subscribedTopicIds(),
new Assignment(Collections.unmodifiableMap(memberAssignment.partitions()))
new Assignment(Map.copyOf(memberAssignment.partitions()))
));
}

View File

@ -47,7 +47,6 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -114,7 +113,7 @@ public class ShareGroupAssignorBenchmark {
private GroupSpec groupSpec;
private List<String> allTopicNames = Collections.emptyList();
private List<String> allTopicNames = List.of();
private TopicIds.TopicResolver topicResolver;
@ -179,14 +178,14 @@ public class ShareGroupAssignorBenchmark {
for (String memberId : groupSpec.memberIds()) {
MemberAssignment memberAssignment = members.getOrDefault(
memberId,
new MemberAssignmentImpl(Collections.emptyMap())
new MemberAssignmentImpl(Map.of())
);
updatedMemberSpec.put(memberId, new MemberSubscriptionAndAssignmentImpl(
groupSpec.memberSubscription(memberId).rackId(),
Optional.empty(),
groupSpec.memberSubscription(memberId).subscribedTopicIds(),
new Assignment(Collections.unmodifiableMap(memberAssignment.partitions()))
new Assignment(Map.copyOf(memberAssignment.partitions()))
));
}

View File

@ -45,7 +45,6 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Threads;
import org.openjdk.jmh.annotations.Warmup;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -90,7 +89,7 @@ public class TargetAssignmentBuilderBenchmark {
private Map<Uuid, Map<Integer, String>> invertedTargetAssignment;
private List<String> allTopicNames = Collections.emptyList();
private List<String> allTopicNames = List.of();
private MetadataImage metadataImage;

View File

@ -38,12 +38,10 @@ import org.openjdk.jmh.runner.Runner;
import org.openjdk.jmh.runner.RunnerException;
import org.openjdk.jmh.runner.options.OptionsBuilder;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@State(Scope.Benchmark)
@ -66,14 +64,14 @@ public class MetadataResponseBenchmark {
new TopicPartition("benchmark", 42),
Optional.of(4),
Optional.of(42),
IntStream.range(0, nodes).boxed().collect(Collectors.toList()),
IntStream.range(0, nodes).filter(i1 -> i1 % 3 != 0).boxed().collect(Collectors.toList()),
IntStream.range(0, nodes).filter(i2 -> i2 % 3 == 0).boxed().collect(Collectors.toList()));
IntStream.range(0, nodes).boxed().toList(),
IntStream.range(0, nodes).filter(i1 -> i1 % 3 != 0).boxed().toList(),
IntStream.range(0, nodes).filter(i2 -> i2 % 3 == 0).boxed().toList());
nodesById = new HashMap<>(nodes);
for (int i = 0; i < nodes; i++) {
nodesById.put(i, new Node(i, "localhost", 1234));
}
nodesById = Collections.unmodifiableMap(nodesById);
nodesById = Map.copyOf(nodesById);
}
@Benchmark

View File

@ -36,7 +36,7 @@ import org.openjdk.jmh.infra.BenchmarkParams;
import org.openjdk.jmh.infra.Blackhole;
import java.nio.charset.Charset;
import java.util.Collections;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@ -426,7 +426,7 @@ public class JsonConverterBenchmark {
public void setup(BenchmarkParams params) {
converter = new JsonConverter(Boolean.parseBoolean(params.getParam("blackbirdModule")));
converter.configure(Collections.emptyMap(), false);
converter.configure(Map.of(), false);
}
@Benchmark

View File

@ -38,9 +38,8 @@ import org.openjdk.jmh.annotations.Warmup;
import org.openjdk.jmh.infra.Blackhole;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import java.util.function.Function;
@ -106,12 +105,12 @@ public class ValuesBenchmark {
new SchemaAndValue(Schema.STRING_SCHEMA, "{\"1\": 2, \"3\": 4}"),
new SchemaAndValue(SchemaBuilder.array(Schema.INT16_SCHEMA), new short[]{1, 2, 3}),
new SchemaAndValue(SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.BOOLEAN_SCHEMA),
Collections.singletonMap("key", true)),
Map.of("key", true)),
new SchemaAndValue(STRUCT_SCHEMA, new Struct(STRUCT_SCHEMA)
.put("first", 1)
.put("second", "foo")
.put("array", Arrays.asList(1, 2, 3))
.put("map", Collections.singletonMap(1, "value"))
.put("array", List.of(1, 2, 3))
.put("map", Map.of(1, "value"))
.put("nested", new Struct(FLAT_STRUCT_SCHEMA).put("field", 12))),
};

View File

@ -72,14 +72,14 @@ public class TestPurgatoryPerformance {
new DelayedOperationPurgatory<>("fake purgatory", 0, 1000);
CompletionQueue queue = new CompletionQueue();
List<String> gcNames = gcMXBeans.stream().map(MemoryManagerMXBean::getName).collect(Collectors.toList());
List<String> gcNames = gcMXBeans.stream().map(MemoryManagerMXBean::getName).toList();
CountDownLatch latch = new CountDownLatch(numRequests);
long initialCpuTimeNano = getProcessCpuTimeNanos(osMXBean).orElseThrow();
long start = System.currentTimeMillis();
Random rand = new Random();
List<FakeOperationKey> keys = IntStream.range(0, numKeys)
.mapToObj(i -> new FakeOperationKey(format("fakeKey%d", rand.nextInt(numPossibleKeys))))
.collect(Collectors.toList());
.toList();
AtomicLong requestArrivalTime = new AtomicLong(start);
AtomicLong end = new AtomicLong(0);
@ -313,7 +313,7 @@ public class TestPurgatoryPerformance {
}
public void printStats() {
List<Long> samples = this.samples.stream().sorted().collect(Collectors.toList());
List<Long> samples = this.samples.stream().sorted().toList();
long p75 = samples.get((int) (samples.size() * 0.75d));
long p50 = samples.get((int) (samples.size() * 0.5d));

View File

@ -82,8 +82,6 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.File;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@ -131,7 +129,7 @@ public class ReplicaFetcherThreadBenchmark {
List<File> logDirs = config.logDirs().stream().map(File::new).toList();
logManager = new LogManagerBuilder().
setLogDirs(logDirs).
setInitialOfflineDirs(Collections.emptyList()).
setInitialOfflineDirs(List.of()).
setConfigRepository(new MockConfigRepository()).
setInitialDefaultConfig(logConfig).
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)).
@ -165,7 +163,7 @@ public class ReplicaFetcherThreadBenchmark {
for (int i = 0; i < partitionCount; i++) {
TopicPartition tp = new TopicPartition("topic", i);
List<Integer> replicas = Arrays.asList(0, 1, 2);
List<Integer> replicas = List.of(0, 1, 2);
PartitionState partitionState = new PartitionState()
.setLeader(0)
.setLeaderEpoch(0)

View File

@ -82,8 +82,6 @@ import org.openjdk.jmh.annotations.TearDown;
import org.openjdk.jmh.annotations.Warmup;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
@ -141,7 +139,7 @@ public class KRaftMetadataRequestBenchmark {
MetadataDelta buildupMetadataDelta = new MetadataDelta(MetadataImage.EMPTY);
IntStream.range(0, 5).forEach(brokerId -> {
RegisterBrokerRecord.BrokerEndpointCollection endpoints = new RegisterBrokerRecord.BrokerEndpointCollection();
endpoints(brokerId).forEach(endpoint -> endpoints.add(endpoint));
endpoints.addAll(endpoints(brokerId));
buildupMetadataDelta.replay(new RegisterBrokerRecord().
setBrokerId(brokerId).
setBrokerEpoch(100L).
@ -157,10 +155,10 @@ public class KRaftMetadataRequestBenchmark {
buildupMetadataDelta.replay(new PartitionRecord().
setPartitionId(partitionId).
setTopicId(topicId).
setReplicas(Arrays.asList(0, 1, 3)).
setIsr(Arrays.asList(0, 1, 3)).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setReplicas(List.of(0, 1, 3)).
setIsr(List.of(0, 1, 3)).
setRemovingReplicas(List.of()).
setAddingReplicas(List.of()).
setLeader(partitionCount % 5).
setLeaderEpoch(0)));
});
@ -168,7 +166,7 @@ public class KRaftMetadataRequestBenchmark {
}
private List<RegisterBrokerRecord.BrokerEndpoint> endpoints(final int brokerId) {
return Collections.singletonList(
return List.of(
new RegisterBrokerRecord.BrokerEndpoint().
setHost("host_" + brokerId).
setPort(9092).

View File

@ -37,7 +37,7 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@ -77,8 +77,8 @@ public class TopicsImageSingleRecordChangeBenchmark {
setTopicId(newTopicUuid).
setReplicas(replicas).
setIsr(isr).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setRemovingReplicas(List.of()).
setAddingReplicas(List.of()).
setLeader(0);
topicsDelta.replay(newPartitionRecord);
System.out.print("(Adding a single topic to metadata having " + totalTopicCount + " total topics) ");

View File

@ -37,7 +37,7 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.IntStream;
@ -84,8 +84,8 @@ public class TopicsImageSnapshotLoadBenchmark {
setTopicId(topicId).
setReplicas(replicas).
setIsr(isr).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setRemovingReplicas(List.of()).
setAddingReplicas(List.of()).
setLeader(currentLeader.get()));
currentLeader.set((1 + currentLeader.get()) % numBrokers);
});

View File

@ -36,12 +36,10 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
@State(Scope.Benchmark)
@Fork(value = 1)
@ -75,16 +73,16 @@ public class TopicsImageZonalOutageBenchmark {
Set<Uuid> perturbedTopics = new HashSet<>();
builtupTopicsImage.topicsById().forEach((topicId, topicImage) ->
topicImage.partitions().forEach((partitionNumber, partitionRegistration) -> {
List<Integer> newIsr = Arrays.stream(partitionRegistration.isr).boxed().filter(n -> n != 0).collect(Collectors.toList());
List<Integer> newIsr = Arrays.stream(partitionRegistration.isr).boxed().filter(n -> n != 0).toList();
if (newIsr.size() < replicationFactor) {
perturbedTopics.add(topicId);
topicsDelta.replay(new PartitionRecord().
setPartitionId(partitionNumber).
setTopicId(topicId).
setReplicas(Arrays.stream(partitionRegistration.replicas).boxed().collect(Collectors.toList())).
setReplicas(Arrays.stream(partitionRegistration.replicas).boxed().toList()).
setIsr(newIsr).
setRemovingReplicas(Collections.emptyList()).
setAddingReplicas(Collections.emptyList()).
setRemovingReplicas(List.of()).
setAddingReplicas(List.of()).
setLeader(newIsr.get(0)));
}
})

View File

@ -58,8 +58,6 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
@ -80,7 +78,7 @@ import scala.jdk.javaapi.OptionConverters;
public class PartitionMakeFollowerBenchmark {
private final File logDir = new File(System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString());
private final KafkaScheduler scheduler = new KafkaScheduler(1, true, "scheduler");
private final List<Integer> replicas = Arrays.asList(0, 1, 2);
private final List<Integer> replicas = List.of(0, 1, 2);
private final OffsetCheckpoints offsetCheckpoints = Mockito.mock(OffsetCheckpoints.class);
private final DelayedOperations delayedOperations = Mockito.mock(DelayedOperations.class);
private final ExecutorService executorService = Executors.newSingleThreadExecutor();
@ -99,8 +97,8 @@ public class PartitionMakeFollowerBenchmark {
BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false);
LogDirFailureChannel logDirFailureChannel = Mockito.mock(LogDirFailureChannel.class);
logManager = new LogManagerBuilder().
setLogDirs(Collections.singletonList(logDir)).
setInitialOfflineDirs(Collections.emptyList()).
setLogDirs(List.of(logDir)).
setInitialOfflineDirs(List.of()).
setConfigRepository(new MockConfigRepository()).
setInitialDefaultConfig(logConfig).
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)).

View File

@ -55,7 +55,6 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.File;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
@ -89,8 +88,8 @@ public class UpdateFollowerFetchStateBenchmark {
scheduler.startup();
LogConfig logConfig = createLogConfig();
logManager = new LogManagerBuilder().
setLogDirs(Collections.singletonList(logDir)).
setInitialOfflineDirs(Collections.emptyList()).
setLogDirs(List.of(logDir)).
setInitialOfflineDirs(List.of()).
setConfigRepository(new MockConfigRepository()).
setInitialDefaultConfig(logConfig).
setCleanerConfig(new CleanerConfig(0, 0, 0, 0, 0, 0.0, 0, false)).

View File

@ -37,10 +37,8 @@ import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.annotations.Warmup;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@State(Scope.Benchmark)
@ -52,15 +50,15 @@ import java.util.stream.IntStream;
public class ProducerRequestBenchmark {
private static final int NUMBER_OF_PARTITIONS = 3;
private static final int NUMBER_OF_RECORDS = 3;
private static final List<ProduceRequestData.TopicProduceData> TOPIC_PRODUCE_DATA = Collections.singletonList(new ProduceRequestData.TopicProduceData()
private static final List<ProduceRequestData.TopicProduceData> TOPIC_PRODUCE_DATA = List.of(new ProduceRequestData.TopicProduceData()
.setTopicId(Uuid.fromString("4NeOmt3TH3vW7AKKORPaCW"))
.setPartitionData(IntStream.range(0, NUMBER_OF_PARTITIONS).mapToObj(partitionIndex -> new ProduceRequestData.PartitionProduceData()
.setIndex(partitionIndex)
.setRecords(MemoryRecords.withRecords(Compression.NONE, IntStream.range(0, NUMBER_OF_RECORDS)
.mapToObj(recordIndex -> new SimpleRecord(100, "hello0".getBytes(StandardCharsets.UTF_8)))
.collect(Collectors.toList())
.toList()
.toArray(new SimpleRecord[0]))))
.collect(Collectors.toList()))
.toList())
);
private static final ProduceRequestData PRODUCE_REQUEST_DATA = new ProduceRequestData()
.setTimeoutMs(100)

View File

@ -60,7 +60,7 @@ public class ProducerResponseBenchmark {
0,
IntStream.range(0, NUMBER_OF_RECORDS)
.mapToObj(ProduceResponse.RecordError::new)
.collect(Collectors.toList()))
.toList())
))
.collect(Collectors.toMap(AbstractMap.SimpleEntry::getKey, AbstractMap.SimpleEntry::getValue));

View File

@ -60,7 +60,6 @@ import org.openjdk.jmh.annotations.Warmup;
import java.io.File;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
@ -123,7 +122,7 @@ public class PartitionCreationBench {
ConfigRepository configRepository = new MockConfigRepository();
this.logManager = new LogManagerBuilder().
setLogDirs(files).
setInitialOfflineDirs(Collections.emptyList()).
setInitialOfflineDirs(List.of()).
setConfigRepository(configRepository).
setInitialDefaultConfig(createLogConfig()).
setCleanerConfig(cleanerConfig).