diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokersToElrs.java b/metadata/src/main/java/org/apache/kafka/controller/BrokersToElrs.java index 656ba7eb8ec..022e3ab3fad 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokersToElrs.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokersToElrs.java @@ -23,7 +23,6 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; import java.util.Arrays; -import java.util.Collections; import java.util.Map; import static org.apache.kafka.metadata.Replicas.NONE; @@ -154,7 +153,7 @@ public class BrokersToElrs { BrokersToIsrs.PartitionsOnReplicaIterator partitionsWithBrokerInElr(int brokerId) { Map topicMap = elrMembers.get(brokerId); if (topicMap == null) { - topicMap = Collections.emptyMap(); + topicMap = Map.of(); } return new BrokersToIsrs.PartitionsOnReplicaIterator(topicMap, false); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java index 5e871b5cb52..24d3ed62b02 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java +++ b/metadata/src/main/java/org/apache/kafka/controller/BrokersToIsrs.java @@ -24,7 +24,6 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.TimelineHashMap; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; import java.util.Map; import java.util.Map.Entry; @@ -270,7 +269,7 @@ public class BrokersToIsrs { PartitionsOnReplicaIterator iterator(int brokerId, boolean leadersOnly) { Map topicMap = isrMembers.get(brokerId); if (topicMap == null) { - topicMap = Collections.emptyMap(); + topicMap = Map.of(); } return new PartitionsOnReplicaIterator(topicMap, leadersOnly); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java index 306db52fdb9..3f9b3f61f10 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ClientQuotaControlManager.java @@ -38,7 +38,6 @@ import java.net.InetAddress; import java.net.UnknownHostException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -182,7 +181,7 @@ public class ClientQuotaControlManager { List newRecords = new ArrayList<>(newQuotaConfigs.size()); Map currentQuotas = clientQuotaData.containsKey(entity) ? - clientQuotaData.get(entity) : Collections.emptyMap(); + clientQuotaData.get(entity) : Map.of(); for (Map.Entry entry : newQuotaConfigs.entrySet()) { String key = entry.getKey(); Double newValue = entry.getValue(); diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index b8a0a0ad425..5ea4b3b6a4e 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -44,7 +44,6 @@ import org.slf4j.Logger; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -85,7 +84,7 @@ public class ConfigurationControlManager { private Consumer existenceChecker = __ -> { }; private Optional alterConfigPolicy = Optional.empty(); private ConfigurationValidator validator = ConfigurationValidator.NO_OP; - private Map staticConfig = Collections.emptyMap(); + private Map staticConfig = Map.of(); private int nodeId = 0; private FeatureControlManager featureControl = null; @@ -174,7 +173,7 @@ public class ConfigurationControlManager { this.validator = validator; this.configData = new TimelineHashMap<>(snapshotRegistry, 0); this.brokersWithConfigs = new TimelineHashSet<>(snapshotRegistry, 0); - this.staticConfig = Collections.unmodifiableMap(new HashMap<>(staticConfig)); + this.staticConfig = Map.copyOf(staticConfig); this.currentController = new ConfigResource(Type.BROKER, Integer.toString(nodeId)); this.featureControl = featureControl; } @@ -217,7 +216,7 @@ public class ConfigurationControlManager { List createClearElrRecordsAsNeeded(List input) { if (!featureControl.isElrFeatureEnabled()) { - return Collections.emptyList(); + return List.of(); } List output = new ArrayList<>(); for (ApiMessageAndVersion messageAndVersion : input) { @@ -309,7 +308,7 @@ public class ConfigurationControlManager { setValue(newValue), (short) 0)); } } - ApiError error = validateAlterConfig(configResource, newRecords, Collections.emptyList(), newlyCreatedResource); + ApiError error = validateAlterConfig(configResource, newRecords, List.of(), newlyCreatedResource); if (error.isFailure()) { return error; } @@ -452,7 +451,7 @@ public class ConfigurationControlManager { List recordsExplicitlyAltered = new ArrayList<>(); Map currentConfigs = configData.get(configResource); if (currentConfigs == null) { - currentConfigs = Collections.emptyMap(); + currentConfigs = Map.of(); } for (Entry entry : newConfigs.entrySet()) { String key = entry.getKey(); @@ -544,7 +543,7 @@ public class ConfigurationControlManager { Map getConfigs(ConfigResource configResource) { Map map = configData.get(configResource); if (map == null) { - return Collections.emptyMap(); + return Map.of(); } else { return Map.copyOf(map); } @@ -711,17 +710,17 @@ public class ConfigurationControlManager { Map clusterConfig() { Map result = configData.get(DEFAULT_NODE); - return (result == null) ? Collections.emptyMap() : result; + return (result == null) ? Map.of() : result; } Map currentControllerConfig() { Map result = configData.get(currentController); - return (result == null) ? Collections.emptyMap() : result; + return (result == null) ? Map.of() : result; } Map currentTopicConfig(String topicName) { Map result = configData.get(new ConfigResource(Type.TOPIC, topicName)); - return (result == null) ? Collections.emptyMap() : result; + return (result == null) ? Map.of() : result; } // Visible to test diff --git a/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java index 32fe419b6eb..ad445fecb82 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ControllerResult.java @@ -19,7 +19,6 @@ package org.apache.kafka.controller; import org.apache.kafka.server.common.ApiMessageAndVersion; -import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.stream.Collectors; @@ -76,7 +75,7 @@ class ControllerResult { } public ControllerResult withoutRecords() { - return new ControllerResult<>(Collections.emptyList(), response, false); + return new ControllerResult<>(List.of(), response, false); } public static ControllerResult atomicOf(List records, T response) { diff --git a/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java index e9bfe10436c..050313d156f 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/DelegationTokenControlManager.java @@ -42,8 +42,8 @@ import org.slf4j.Logger; import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Base64; -import java.util.Collections; import java.util.List; +import java.util.Set; import javax.crypto.Mac; import javax.crypto.spec.SecretKeySpec; @@ -98,7 +98,7 @@ public class DelegationTokenControlManager { DelegationTokenControlManager build() { if (logContext == null) logContext = new LogContext(); - if (tokenCache == null) tokenCache = new DelegationTokenCache(Collections.emptySet()); + if (tokenCache == null) tokenCache = new DelegationTokenCache(Set.of()); return new DelegationTokenControlManager( logContext, tokenCache, diff --git a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java index 92c70f44472..c167bb4a31c 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/FeatureControlManager.java @@ -95,7 +95,7 @@ public class FeatureControlManager { localSupportedFeatures.put(MetadataVersion.FEATURE_NAME, VersionRange.of( MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestProduction().featureLevel())); - quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, Collections.singletonList(0)); + quorumFeatures = new QuorumFeatures(0, localSupportedFeatures, List.of(0)); } return new FeatureControlManager( logContext, @@ -157,12 +157,12 @@ public class FeatureControlManager { ApiError error = updateFeature(entry.getKey(), entry.getValue(), upgradeTypes.getOrDefault(entry.getKey(), FeatureUpdate.UpgradeType.UPGRADE), records, proposedUpdatedVersions); if (!error.error().equals(Errors.NONE)) { - return ControllerResult.of(Collections.emptyList(), error); + return ControllerResult.of(List.of(), error); } } if (validateOnly) { - return ControllerResult.of(Collections.emptyList(), ApiError.NONE); + return ControllerResult.of(List.of(), ApiError.NONE); } else { return ControllerResult.atomicOf(records, ApiError.NONE); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java index fe3a55f7593..2fe49220018 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java +++ b/metadata/src/main/java/org/apache/kafka/controller/PartitionChangeBuilder.java @@ -33,7 +33,6 @@ import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -332,7 +331,7 @@ public class PartitionChangeBuilder { // so only log clean elections at TRACE level; log unclean elections at INFO level // to ensure the message is emitted since an unclean election can lead to data loss; if (targetElr.contains(electionResult.node)) { - targetIsr = Collections.singletonList(electionResult.node); + targetIsr = List.of(electionResult.node); targetElr = targetElr.stream().filter(replica -> replica != electionResult.node) .collect(Collectors.toList()); log.trace("Setting new leader for topicId {}, partition {} to {} using ELR", @@ -348,7 +347,7 @@ public class PartitionChangeBuilder { if (electionResult.unclean) { // If the election was unclean, we have to forcibly set the ISR to just the // new leader. This can result in data loss! - record.setIsr(Collections.singletonList(electionResult.node)); + record.setIsr(List.of(electionResult.node)); if (partition.leaderRecoveryState != LeaderRecoveryState.RECOVERING) { // And mark the leader recovery state as RECOVERING record.setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()); @@ -423,8 +422,8 @@ public class PartitionChangeBuilder { targetIsr = completedReassignment.isr; targetReplicas = completedReassignment.replicas; - targetRemoving = Collections.emptyList(); - targetAdding = Collections.emptyList(); + targetRemoving = List.of(); + targetAdding = List.of(); } public Optional build() { @@ -476,7 +475,7 @@ public class PartitionChangeBuilder { for (int replica : targetReplicas) { directories.add(this.targetDirectories.getOrDefault(replica, defaultDirProvider.defaultDir(replica))); } - if (!directories.equals(Arrays.asList(partition.directories))) { + if (!directories.equals(List.of(partition.directories))) { record.setDirectories(directories); } } @@ -497,11 +496,11 @@ public class PartitionChangeBuilder { if (record.isr() != null && record.isr().isEmpty() && (partition.lastKnownElr.length != 1 || partition.lastKnownElr[0] != partition.leader)) { // Only update the last known leader when the first time the partition becomes leaderless. - record.setLastKnownElr(Collections.singletonList(partition.leader)); + record.setLastKnownElr(List.of(partition.leader)); } else if ((record.leader() >= 0 || (partition.leader != NO_LEADER && record.leader() != NO_LEADER)) && partition.lastKnownElr.length > 0) { // Clear the LastKnownElr field if the partition will have or continues to have a valid leader. - record.setLastKnownElr(Collections.emptyList()); + record.setLastKnownElr(List.of()); } } @@ -511,8 +510,8 @@ public class PartitionChangeBuilder { // Clean the ELR related fields if it is an unclean election or ELR is disabled. if (!isCleanLeaderElection || !eligibleLeaderReplicasEnabled) { - targetElr = Collections.emptyList(); - targetLastKnownElr = Collections.emptyList(); + targetElr = List.of(); + targetLastKnownElr = List.of(); } if (!targetElr.equals(Replicas.toList(partition.elr))) { @@ -540,8 +539,8 @@ public class PartitionChangeBuilder { // If the ISR is larger or equal to the min ISR, clear the ELR and LastKnownElr. if (targetIsr.size() >= minISR) { - targetElr = Collections.emptyList(); - targetLastKnownElr = Collections.emptyList(); + targetElr = List.of(); + targetLastKnownElr = List.of(); return; } diff --git a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java index 2c1a0b49e4c..a0d56b90f7b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ProducerIdControlManager.java @@ -28,7 +28,7 @@ import org.apache.kafka.timeline.TimelineObject; import org.slf4j.Logger; -import java.util.Collections; +import java.util.List; public class ProducerIdControlManager { @@ -97,7 +97,7 @@ public class ProducerIdControlManager { .setNextProducerId(newNextProducerId) .setBrokerId(brokerId) .setBrokerEpoch(brokerEpoch); - return ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion(record, (short) 0)), block); + return ControllerResult.of(List.of(new ApiMessageAndVersion(record, (short) 0)), block); } // VisibleForTesting diff --git a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java index c2fbc2ca1a2..56e3848ed32 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java +++ b/metadata/src/main/java/org/apache/kafka/controller/QuorumController.java @@ -128,7 +128,6 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.slf4j.Logger; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.Iterator; @@ -215,7 +214,7 @@ public final class QuorumController implements Controller { private Optional createTopicPolicy = Optional.empty(); private Optional alterConfigPolicy = Optional.empty(); private ConfigurationValidator configurationValidator = ConfigurationValidator.NO_OP; - private Map staticConfig = Collections.emptyMap(); + private Map staticConfig = Map.of(); private BootstrapMetadata bootstrapMetadata = null; private int maxRecordsPerBatch = DEFAULT_MAX_RECORDS_PER_BATCH; private long controllerPerformanceSamplePeriodMs = 60000L; @@ -1703,7 +1702,7 @@ public final class QuorumController implements Controller { periodicControl.registerTask(new PeriodicTask("generatePeriodicPerformanceMessage", () -> { performanceMonitor.generatePeriodicPerformanceMessage(); - return ControllerResult.of(Collections.emptyList(), false); + return ControllerResult.of(List.of(), false); }, performanceMonitor.periodNs(), EnumSet.noneOf(PeriodicTaskFlag.class))); @@ -1802,7 +1801,7 @@ public final class QuorumController implements Controller { Collection names ) { if (names.isEmpty()) - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); return appendReadEvent("findTopicIds", context.deadlineNs(), () -> replicationControl.findTopicIds(offsetControl.lastStableOffset(), names)); } @@ -1821,7 +1820,7 @@ public final class QuorumController implements Controller { Collection ids ) { if (ids.isEmpty()) - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); return appendReadEvent("findTopicNames", context.deadlineNs(), () -> replicationControl.findTopicNames(offsetControl.lastStableOffset(), ids)); } @@ -1832,7 +1831,7 @@ public final class QuorumController implements Controller { Collection ids ) { if (ids.isEmpty()) - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); return appendWriteEvent("deleteTopics", context.deadlineNs(), () -> replicationControl.deleteTopics(context, ids)); } @@ -1875,7 +1874,7 @@ public final class QuorumController implements Controller { boolean validateOnly ) { if (configChanges.isEmpty()) { - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); } return appendWriteEvent("incrementalAlterConfigs", context.deadlineNs(), () -> { ControllerResult> result = @@ -1920,7 +1919,7 @@ public final class QuorumController implements Controller { Map> newConfigs, boolean validateOnly ) { if (newConfigs.isEmpty()) { - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); } return appendWriteEvent("legacyAlterConfigs", context.deadlineNs(), () -> { ControllerResult> result = @@ -2014,7 +2013,7 @@ public final class QuorumController implements Controller { boolean validateOnly ) { if (quotaAlterations.isEmpty()) { - return CompletableFuture.completedFuture(Collections.emptyMap()); + return CompletableFuture.completedFuture(Map.of()); } return appendWriteEvent("alterClientQuotas", context.deadlineNs(), () -> { ControllerResult> result = @@ -2085,7 +2084,7 @@ public final class QuorumController implements Controller { boolean validateOnly ) { if (topics.isEmpty()) { - return CompletableFuture.completedFuture(Collections.emptyList()); + return CompletableFuture.completedFuture(List.of()); } return appendWriteEvent("createPartitions", context.deadlineNs(), () -> { diff --git a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java index ee3fe6a9a96..0b2c8ed544b 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ReplicationControlManager.java @@ -121,7 +121,6 @@ import java.util.OptionalInt; import java.util.Set; import java.util.function.IntPredicate; import java.util.function.Supplier; -import java.util.stream.Collectors; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; @@ -611,8 +610,8 @@ public class ReplicationControlManager { new PartitionChangeRecord(). setPartitionId(partitionId). setTopicId(topic.id). - setEligibleLeaderReplicas(Collections.emptyList()). - setLastKnownElr(Collections.emptyList()))); + setEligibleLeaderReplicas(List.of()). + setLastKnownElr(List.of()))); numRemoved++; } } @@ -660,7 +659,7 @@ public class ReplicationControlManager { configRecords = configResult.records(); } } else { - configRecords = Collections.emptyList(); + configRecords = List.of(); } ApiError error; try { @@ -697,7 +696,7 @@ public class ReplicationControlManager { } if (request.validateOnly()) { log.info("Validate-only CreateTopics result(s): {}", resultsBuilder); - return ControllerResult.atomicOf(Collections.emptyList(), data); + return ControllerResult.atomicOf(List.of(), data); } else { log.info("CreateTopics result(s): {}", resultsBuilder); return ControllerResult.atomicOf(records, data); @@ -734,7 +733,7 @@ public class ReplicationControlManager { validateManualPartitionAssignment(partitionAssignment, replicationFactor); replicationFactor = OptionalInt.of(assignment.brokerIds().size()); List isr = assignment.brokerIds().stream(). - filter(clusterControl::isActive).collect(Collectors.toList()); + filter(clusterControl::isActive).toList(); if (isr.isEmpty()) { return new ApiError(Errors.INVALID_REPLICA_ASSIGNMENT, "All brokers specified in the manual partition assignment for " + @@ -778,7 +777,7 @@ public class ReplicationControlManager { for (int partitionId = 0; partitionId < topicAssignment.assignments().size(); partitionId++) { PartitionAssignment partitionAssignment = topicAssignment.assignments().get(partitionId); List isr = partitionAssignment.replicas().stream(). - filter(clusterControl::isActive).collect(Collectors.toList()); + filter(clusterControl::isActive).toList(); // If the ISR is empty, it means that all brokers are fenced or // in controlled shutdown. To be consistent with the replica placer, // we reject the create topic request with INVALID_REPLICATION_FACTOR. @@ -1673,7 +1672,7 @@ public class ReplicationControlManager { Optional idAndEpoch = heartbeatManager.tracker().maybeRemoveExpired(); if (idAndEpoch.isEmpty()) { log.debug("No stale brokers found."); - return ControllerResult.of(Collections.emptyList(), false); + return ControllerResult.of(List.of(), false); } int id = idAndEpoch.get().id(); long epoch = idAndEpoch.get().epoch(); @@ -1681,12 +1680,12 @@ public class ReplicationControlManager { log.info("Removing heartbeat tracker entry for unknown broker {} at epoch {}.", id, epoch); heartbeatManager.remove(id); - return ControllerResult.of(Collections.emptyList(), true); + return ControllerResult.of(List.of(), true); } else if (clusterControl.brokerRegistrations().get(id).epoch() != epoch) { log.info("Removing heartbeat tracker entry for broker {} at previous epoch {}. " + "Current epoch is {}", id, epoch, clusterControl.brokerRegistrations().get(id).epoch()); - return ControllerResult.of(Collections.emptyList(), true); + return ControllerResult.of(List.of(), true); } // Even though multiple brokers can go stale at a time, we will process // fencing one at a time so that the effect of fencing each broker is visible @@ -1887,7 +1886,7 @@ public class ReplicationControlManager { validateManualPartitionAssignment(partitionAssignment, OptionalInt.of(replicationFactor)); partitionAssignments.add(partitionAssignment); List isr = partitionAssignment.replicas().stream(). - filter(clusterControl::isActive).collect(Collectors.toList()); + filter(clusterControl::isActive).toList(); if (isr.isEmpty()) { throw new InvalidReplicaAssignmentException( "All brokers specified in the manual partition assignment for " + @@ -1900,13 +1899,13 @@ public class ReplicationControlManager { new PlacementSpec(startPartitionId, additional, replicationFactor), clusterDescriber ).assignments(); - isrs = partitionAssignments.stream().map(PartitionAssignment::replicas).collect(Collectors.toList()); + isrs = partitionAssignments.stream().map(PartitionAssignment::replicas).toList(); } int partitionId = startPartitionId; for (int i = 0; i < partitionAssignments.size(); i++) { PartitionAssignment partitionAssignment = partitionAssignments.get(i); List isr = isrs.get(i).stream(). - filter(clusterControl::isActive).collect(Collectors.toList()); + filter(clusterControl::isActive).toList(); // If the ISR is empty, it means that all brokers are fenced or // in controlled shutdown. To be consistent with the replica placer, // we reject the create topic request with INVALID_REPLICATION_FACTOR. @@ -2025,7 +2024,7 @@ public class ReplicationControlManager { builder.setElection(PartitionChangeBuilder.Election.UNCLEAN); } if (brokerWithUncleanShutdown != NO_LEADER) { - builder.setUncleanShutdownReplicas(Collections.singletonList(brokerWithUncleanShutdown)); + builder.setUncleanShutdownReplicas(List.of(brokerWithUncleanShutdown)); } // Note: if brokerToRemove and brokerWithUncleanShutdown were passed as NO_LEADER, this is a no-op (the new @@ -2148,8 +2147,8 @@ public class ReplicationControlManager { return builder .setTargetIsr(revert.isr()). setTargetReplicas(revert.replicas()). - setTargetRemoving(Collections.emptyList()). - setTargetAdding(Collections.emptyList()). + setTargetRemoving(List.of()). + setTargetAdding(List.of()). setDefaultDirProvider(clusterDescriber). build(); } diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java index 1c7c46ae8b0..938bf1ed48a 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/ControllerMetadataMetrics.java @@ -24,7 +24,7 @@ import com.yammer.metrics.core.Meter; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; -import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -223,7 +223,7 @@ public final class ControllerMetadataMetrics implements AutoCloseable { @Override public void close() { - registry.ifPresent(r -> Arrays.asList( + registry.ifPresent(r -> List.of( FENCED_BROKER_COUNT, ACTIVE_BROKER_COUNT, GLOBAL_TOPIC_COUNT, diff --git a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java index 10c3807da85..c4dafa898ad 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/controller/metrics/QuorumControllerMetrics.java @@ -25,7 +25,7 @@ import com.yammer.metrics.core.Histogram; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; -import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -222,7 +222,7 @@ public class QuorumControllerMetrics implements AutoCloseable { @Override public void close() { - registry.ifPresent(r -> Arrays.asList( + registry.ifPresent(r -> List.of( ACTIVE_CONTROLLER_COUNT, EVENT_QUEUE_TIME_MS, EVENT_QUEUE_PROCESSING_TIME_MS, diff --git a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java index c2fe142d71f..12f778b841e 100644 --- a/metadata/src/main/java/org/apache/kafka/image/AclsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/AclsImage.java @@ -34,7 +34,7 @@ import java.util.Map.Entry; * This class is thread-safe. */ public final class AclsImage { - public static final AclsImage EMPTY = new AclsImage(Collections.emptyMap()); + public static final AclsImage EMPTY = new AclsImage(Map.of()); private final Map acls; diff --git a/metadata/src/main/java/org/apache/kafka/image/ClientQuotaImage.java b/metadata/src/main/java/org/apache/kafka/image/ClientQuotaImage.java index 00f56fef53d..3e139925eb1 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClientQuotaImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClientQuotaImage.java @@ -40,7 +40,7 @@ import java.util.Objects; * This class is thread-safe. */ public final class ClientQuotaImage { - public static final ClientQuotaImage EMPTY = new ClientQuotaImage(Collections.emptyMap()); + public static final ClientQuotaImage EMPTY = new ClientQuotaImage(Map.of()); private final Map quotas; diff --git a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java index 7ad86fda5e2..77096e5d009 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClientQuotasImage.java @@ -50,7 +50,7 @@ import static org.apache.kafka.common.requests.DescribeClientQuotasRequest.MATCH * This class is thread-safe. */ public final class ClientQuotasImage { - public static final ClientQuotasImage EMPTY = new ClientQuotasImage(Collections.emptyMap()); + public static final ClientQuotasImage EMPTY = new ClientQuotasImage(Map.of()); private final Map entities; diff --git a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java index 2e8951526ba..713e62fadc6 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ClusterImage.java @@ -35,8 +35,8 @@ import java.util.Objects; */ public final class ClusterImage { public static final ClusterImage EMPTY = new ClusterImage( - Collections.emptyMap(), - Collections.emptyMap()); + Map.of(), + Map.of()); private final Map brokers; diff --git a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java index 9d794bfddc2..0b3fcbb3867 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsDelta.java @@ -23,7 +23,6 @@ import org.apache.kafka.common.metadata.ConfigRecord; import org.apache.kafka.common.metadata.RemoveTopicRecord; import org.apache.kafka.server.common.MetadataVersion; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; @@ -62,7 +61,7 @@ public final class ConfigurationsDelta { ConfigResource resource = new ConfigResource(Type.forId(record.resourceType()), record.resourceName()); ConfigurationImage configImage = image.resourceData().getOrDefault(resource, - new ConfigurationImage(resource, Collections.emptyMap())); + new ConfigurationImage(resource, Map.of())); ConfigurationDelta delta = changes.computeIfAbsent(resource, __ -> new ConfigurationDelta(configImage)); delta.replay(record); diff --git a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsImage.java b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsImage.java index 20df483fecc..4df6c363840 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ConfigurationsImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ConfigurationsImage.java @@ -36,7 +36,7 @@ import java.util.Properties; */ public final class ConfigurationsImage { public static final ConfigurationsImage EMPTY = - new ConfigurationsImage(Collections.emptyMap()); + new ConfigurationsImage(Map.of()); private final Map data; @@ -70,7 +70,7 @@ public final class ConfigurationsImage { if (configurationImage != null) { return configurationImage.toMap(); } else { - return Collections.emptyMap(); + return Map.of(); } } diff --git a/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java b/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java index 52a4fd911c6..c2c90a4bbb9 100644 --- a/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/DelegationTokenImage.java @@ -35,7 +35,7 @@ import java.util.Map.Entry; * This class is thread-safe. */ public final class DelegationTokenImage { - public static final DelegationTokenImage EMPTY = new DelegationTokenImage(Collections.emptyMap()); + public static final DelegationTokenImage EMPTY = new DelegationTokenImage(Map.of()); // Map TokenID to TokenInformation. // The TokenID is also contained in the TokenInformation inside the DelegationTokenData diff --git a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java index a8ca48ad731..547a4417048 100644 --- a/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/FeaturesImage.java @@ -38,7 +38,7 @@ import java.util.Optional; */ public final class FeaturesImage { public static final FeaturesImage EMPTY = new FeaturesImage( - Collections.emptyMap(), + Map.of(), Optional.empty() ); diff --git a/metadata/src/main/java/org/apache/kafka/image/ScramImage.java b/metadata/src/main/java/org/apache/kafka/image/ScramImage.java index cfe489d33d2..b6dedc84625 100644 --- a/metadata/src/main/java/org/apache/kafka/image/ScramImage.java +++ b/metadata/src/main/java/org/apache/kafka/image/ScramImage.java @@ -43,7 +43,7 @@ import java.util.Map.Entry; * This class is thread-safe. */ public final class ScramImage { - public static final ScramImage EMPTY = new ScramImage(Collections.emptyMap()); + public static final ScramImage EMPTY = new ScramImage(Map.of()); private final Map> mechanisms; diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java index a663cbfd7d9..00e4a422341 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java @@ -26,9 +26,9 @@ import org.apache.kafka.common.metadata.PartitionRecord; import org.apache.kafka.metadata.PartitionRegistration; import org.apache.kafka.metadata.Replicas; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -104,8 +104,8 @@ public final class TopicDelta { new PartitionChangeRecord(). setPartitionId(partitionId). setTopicId(image.id()). - setEligibleLeaderReplicas(Collections.emptyList()). - setLastKnownElr(Collections.emptyList()) + setEligibleLeaderReplicas(List.of()). + setLastKnownElr(List.of()) )); } } diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java index ec5cee135b3..b6deed5281a 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicsDelta.java @@ -30,7 +30,6 @@ import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.immutable.ImmutableMap; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -72,7 +71,7 @@ public final class TopicsDelta { public void replay(TopicRecord record) { TopicDelta delta = new TopicDelta( - new TopicImage(record.name(), record.topicId(), Collections.emptyMap())); + new TopicImage(record.name(), record.topicId(), Map.of())); changedTopics.put(record.topicId(), delta); createdTopics.put(record.name(), record.topicId()); } diff --git a/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java b/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java index feaf387d4c3..6fc346ed10a 100644 --- a/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetrics.java @@ -25,7 +25,7 @@ import com.yammer.metrics.core.Gauge; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; -import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; @@ -144,7 +144,7 @@ public final class MetadataLoaderMetrics implements AutoCloseable { @Override public void close() { - registry.ifPresent(r -> Arrays.asList( + registry.ifPresent(r -> List.of( CURRENT_METADATA_VERSION, CURRENT_CONTROLLER_ID, HANDLE_LOAD_SNAPSHOT_COUNT diff --git a/metadata/src/main/java/org/apache/kafka/image/node/AclsImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/AclsImageNode.java index 8152dbd8336..d4a0a087dfd 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/AclsImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/AclsImageNode.java @@ -20,7 +20,7 @@ package org.apache.kafka.image.node; import org.apache.kafka.image.AclsImage; import java.util.Collection; -import java.util.Collections; +import java.util.List; public class AclsImageNode implements MetadataNode { @@ -40,7 +40,7 @@ public class AclsImageNode implements MetadataNode { @Override public Collection childNames() { - return Collections.singletonList(AclsImageByIdNode.NAME); + return List.of(AclsImageByIdNode.NAME); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/image/node/ClusterImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/ClusterImageNode.java index e631c0ef6d9..9ee77ada875 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/ClusterImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/ClusterImageNode.java @@ -19,8 +19,8 @@ package org.apache.kafka.image.node; import org.apache.kafka.image.ClusterImage; -import java.util.Arrays; import java.util.Collection; +import java.util.List; public class ClusterImageNode implements MetadataNode { @@ -40,7 +40,7 @@ public class ClusterImageNode implements MetadataNode { @Override public Collection childNames() { - return Arrays.asList(ClusterImageBrokersNode.NAME, ClusterImageControllersNode.NAME); + return List.of(ClusterImageBrokersNode.NAME, ClusterImageControllersNode.NAME); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java b/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java index 6e00c214d0a..2139c89131e 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/MetadataNode.java @@ -22,7 +22,7 @@ import org.apache.kafka.image.node.printer.NodeStringifier; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; +import java.util.List; public interface MetadataNode { @@ -34,7 +34,7 @@ public interface MetadataNode { * Get the names of the children of this node, if there are any. */ default Collection childNames() { - return Collections.emptyList(); + return List.of(); } /** diff --git a/metadata/src/main/java/org/apache/kafka/image/node/ProducerIdsImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/ProducerIdsImageNode.java index 91b9457b536..3bb59c30d06 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/ProducerIdsImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/ProducerIdsImageNode.java @@ -20,7 +20,7 @@ package org.apache.kafka.image.node; import org.apache.kafka.image.ProducerIdsImage; import java.util.Collection; -import java.util.Collections; +import java.util.List; public class ProducerIdsImageNode implements MetadataNode { @@ -40,7 +40,7 @@ public class ProducerIdsImageNode implements MetadataNode { @Override public Collection childNames() { - return Collections.singletonList("nextProducerId"); + return List.of("nextProducerId"); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/image/node/ScramImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/ScramImageNode.java index fcae7984f81..6e7bdd731dc 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/ScramImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/ScramImageNode.java @@ -23,7 +23,6 @@ import org.apache.kafka.metadata.ScramCredentialData; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Map; @@ -58,6 +57,6 @@ public class ScramImageNode implements MetadataNode { ScramMechanism mechanism = ScramMechanism.fromMechanismName(name); if (mechanism.equals(ScramMechanism.UNKNOWN)) return null; Map userData = image.mechanisms().get(mechanism); - return new ScramMechanismNode(userData == null ? Collections.emptyMap() : userData); + return new ScramMechanismNode(userData == null ? Map.of() : userData); } } diff --git a/metadata/src/main/java/org/apache/kafka/image/node/TopicsImageNode.java b/metadata/src/main/java/org/apache/kafka/image/node/TopicsImageNode.java index b01f927c6fd..5d48d203370 100644 --- a/metadata/src/main/java/org/apache/kafka/image/node/TopicsImageNode.java +++ b/metadata/src/main/java/org/apache/kafka/image/node/TopicsImageNode.java @@ -19,8 +19,8 @@ package org.apache.kafka.image.node; import org.apache.kafka.image.TopicsImage; -import java.util.Arrays; import java.util.Collection; +import java.util.List; public class TopicsImageNode implements MetadataNode { @@ -40,7 +40,7 @@ public class TopicsImageNode implements MetadataNode { @Override public Collection childNames() { - return Arrays.asList(TopicsImageByNameNode.NAME, TopicsImageByIdNode.NAME); + return List.of(TopicsImageByNameNode.NAME, TopicsImageByIdNode.NAME); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisher.java b/metadata/src/main/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisher.java index d04d308d063..45a314e5d54 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisher.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisher.java @@ -26,7 +26,6 @@ import org.apache.kafka.image.loader.LoaderManifest; import org.apache.kafka.image.loader.LoaderManifestType; import org.apache.kafka.metadata.ControllerRegistration; -import java.util.Collections; import java.util.Map; @@ -37,7 +36,7 @@ public class ControllerRegistrationsPublisher implements MetadataPublisher { private volatile Map controllers; public ControllerRegistrationsPublisher() { - this.controllers = Collections.emptyMap(); + this.controllers = Map.of(); } @Override diff --git a/metadata/src/main/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetrics.java b/metadata/src/main/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetrics.java index 87d0ef3474b..560556166c4 100644 --- a/metadata/src/main/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetrics.java +++ b/metadata/src/main/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetrics.java @@ -24,7 +24,7 @@ import com.yammer.metrics.core.Gauge; import com.yammer.metrics.core.MetricName; import com.yammer.metrics.core.MetricsRegistry; -import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -91,7 +91,7 @@ public final class SnapshotEmitterMetrics implements AutoCloseable { @Override public void close() { - registry.ifPresent(r -> Arrays.asList( + registry.ifPresent(r -> List.of( LATEST_SNAPSHOT_GENERATED_BYTES, LATEST_SNAPSHOT_GENERATED_AGE_MS ).forEach(r::removeMetric)); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java index 1a473ad5599..799d486743e 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/BrokerRegistration.java @@ -64,7 +64,7 @@ public class BrokerRegistration { this.fenced = false; this.inControlledShutdown = false; this.isMigratingZkBroker = false; - this.directories = Collections.emptyList(); + this.directories = List.of(); } public Builder setId(int id) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java index 3c00390f71f..f61c4c70e06 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/KafkaConfigSchema.java @@ -33,8 +33,6 @@ import java.util.Map; import java.util.Objects; import java.util.function.Function; -import static java.util.Collections.emptyList; -import static java.util.Collections.emptyMap; import static org.apache.kafka.common.config.TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG; @@ -44,7 +42,7 @@ import static org.apache.kafka.common.config.TopicConfig.MIN_IN_SYNC_REPLICAS_CO * determining the type of config keys (string, int, password, etc.) */ public class KafkaConfigSchema { - public static final KafkaConfigSchema EMPTY = new KafkaConfigSchema(emptyMap(), emptyMap()); + public static final KafkaConfigSchema EMPTY = new KafkaConfigSchema(Map.of(), Map.of()); private static final ConfigDef EMPTY_CONFIG_DEF = new ConfigDef(); @@ -203,7 +201,7 @@ public class KafkaConfigSchema { dynamicTopicConfigs.get(configKey.name), ConfigSource.DYNAMIC_TOPIC_CONFIG, Function.identity()); } - List synonyms = logConfigSynonyms.getOrDefault(configKey.name, emptyList()); + List synonyms = logConfigSynonyms.getOrDefault(configKey.name, List.of()); for (ConfigSynonym synonym : synonyms) { if (dynamicNodeConfigs.containsKey(synonym.name())) { return toConfigEntry(configKey, dynamicNodeConfigs.get(synonym.name()), @@ -233,7 +231,7 @@ public class KafkaConfigSchema { ConfigDef configDef = configDefs.getOrDefault(ConfigResource.Type.BROKER, EMPTY_CONFIG_DEF); ConfigDef.ConfigKey configKey = configDef.configKeys().get(configName); if (configKey == null) return null; - List synonyms = logConfigSynonyms.getOrDefault(configKey.name, emptyList()); + List synonyms = logConfigSynonyms.getOrDefault(configKey.name, List.of()); for (ConfigSynonym synonym : synonyms) { if (staticNodeConfig.containsKey(synonym.name())) { return toConfigEntry(configKey, staticNodeConfig.get(synonym.name()), @@ -278,7 +276,7 @@ public class KafkaConfigSchema { source, configKey.type().isSensitive(), false, // "readonly" is always false, for now. - emptyList(), // we don't populate synonyms, for now. + List.of(), // we don't populate synonyms, for now. translateConfigType(configKey.type()), configKey.documentation); } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/LeaderAndIsr.java b/metadata/src/main/java/org/apache/kafka/metadata/LeaderAndIsr.java index 99d7000bf51..615582870be 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/LeaderAndIsr.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/LeaderAndIsr.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState; import java.util.List; import java.util.Objects; import java.util.Optional; -import java.util.stream.Collectors; public class LeaderAndIsr { public static final int INITIAL_LEADER_EPOCH = 0; @@ -56,7 +55,7 @@ public class LeaderAndIsr { leader, leaderEpoch, leaderRecoveryState, - isr.stream().map(brokerId -> new BrokerState().setBrokerId(brokerId)).collect(Collectors.toList()), + isr.stream().map(brokerId -> new BrokerState().setBrokerId(brokerId)).toList(), partitionEpoch ); } @@ -130,7 +129,7 @@ public class LeaderAndIsr { public List isr() { return isrWithBrokerEpoch.stream() .map(BrokerState::brokerId) - .collect(Collectors.toList()); + .toList(); } public boolean equalsAllowStalePartitionEpoch(LeaderAndIsr other) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java index ce6de613586..a2136e3e2b6 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizer.java @@ -31,7 +31,6 @@ import org.apache.kafka.server.authorizer.AuthorizerServerInfo; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -190,7 +189,7 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer { static Set getConfiguredSuperUsers(Map configs) { Object configValue = configs.get(SUPER_USERS_CONFIG); - if (configValue == null) return Collections.emptySet(); + if (configValue == null) return Set.of(); String[] values = configValue.toString().split(";"); Set result = new HashSet<>(); for (String value : values) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java index ebebeb89ff5..6b139b5d910 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerData.java @@ -36,8 +36,6 @@ import org.apache.kafka.server.authorizer.AuthorizationResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; -import java.util.EnumSet; import java.util.Iterator; import java.util.NavigableSet; import java.util.Set; @@ -122,7 +120,7 @@ public class StandardAuthorizerData { return new StandardAuthorizerData(createLogger(-1), null, false, - Collections.emptySet(), + Set.of(), DENIED, new AclCache()); } @@ -437,14 +435,14 @@ public class StandardAuthorizerData { /** * The set of operations which imply DESCRIBE permission, when used in an ALLOW acl. */ - private static final Set IMPLIES_DESCRIBE = Collections.unmodifiableSet( - EnumSet.of(DESCRIBE, READ, WRITE, DELETE, ALTER)); + private static final Set IMPLIES_DESCRIBE = + Set.of(DESCRIBE, READ, WRITE, DELETE, ALTER); /** * The set of operations which imply DESCRIBE_CONFIGS permission, when used in an ALLOW acl. */ - private static final Set IMPLIES_DESCRIBE_CONFIGS = Collections.unmodifiableSet( - EnumSet.of(DESCRIBE_CONFIGS, ALTER_CONFIGS)); + private static final Set IMPLIES_DESCRIBE_CONFIGS = + Set.of(DESCRIBE_CONFIGS, ALTER_CONFIGS); static AuthorizationResult findResult(Action action, AuthorizableRequestContext requestContext, diff --git a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java index 3cb1aed1794..2402437c663 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadata.java @@ -24,7 +24,6 @@ import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.server.common.MetadataVersion; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Objects; @@ -71,7 +70,7 @@ public class BootstrapMetadata { } public static BootstrapMetadata fromVersion(MetadataVersion metadataVersion, String source) { - List records = Collections.singletonList( + List records = List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(metadataVersion.featureLevel()), (short) 0)); diff --git a/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java b/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java index 3feb3f0c81b..0c597bc7f89 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/placement/StripedReplicaPlacer.java @@ -28,7 +28,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; -import java.util.stream.Collectors; /** @@ -438,7 +437,7 @@ public class StripedReplicaPlacer implements ReplicaPlacer { placements.add(rackList.place(placement.numReplicas())); } return new TopicAssignment( - placements.stream().map(replicas -> new PartitionAssignment(replicas, cluster)).collect(Collectors.toList()) + placements.stream().map(replicas -> new PartitionAssignment(replicas, cluster)).toList() ); } } diff --git a/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java index 600b2d86cc6..73cd33ccf58 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsemble.java @@ -63,9 +63,9 @@ public final class MetaPropertiesEnsemble { /** * A completely empty MetaPropertiesEnsemble object. */ - public static final MetaPropertiesEnsemble EMPTY = new MetaPropertiesEnsemble(Collections.emptySet(), - Collections.emptySet(), - Collections.emptyMap(), + public static final MetaPropertiesEnsemble EMPTY = new MetaPropertiesEnsemble(Set.of(), + Set.of(), + Map.of(), Optional.empty()); /** diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index 95f814e99db..878c0ce042d 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -43,7 +43,6 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -116,7 +115,7 @@ public class Formatter { /** * The arguments passed to --add-scram */ - private List scramArguments = Collections.emptyList(); + private List scramArguments = List.of(); /** * The name of the initial controller listener. diff --git a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java index 9792efee728..af2d3df3150 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java @@ -32,7 +32,6 @@ import java.io.IOException; import java.nio.channels.FileChannel; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.Collections; import java.util.List; import static org.apache.kafka.raft.KafkaRaftClient.MAX_BATCH_SIZE_BYTES; @@ -61,7 +60,7 @@ public class BatchFileWriter implements AutoCloseable { } public void append(ApiMessageAndVersion apiMessageAndVersion) { - batchAccumulator.append(0, Collections.singletonList(apiMessageAndVersion), false); + batchAccumulator.append(0, List.of(apiMessageAndVersion), false); } public void append(List messageBatch) { diff --git a/metadata/src/main/java/org/apache/kafka/metadata/util/SnapshotFileReader.java b/metadata/src/main/java/org/apache/kafka/metadata/util/SnapshotFileReader.java index b712be5add4..2fddf190989 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/util/SnapshotFileReader.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/util/SnapshotFileReader.java @@ -39,7 +39,6 @@ import org.slf4j.LoggerFactory; import java.io.File; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.OptionalInt; @@ -162,7 +161,7 @@ public final class SnapshotFileReader implements AutoCloseable { } listener.handleCommit( MemoryBatchReader.of( - Collections.singletonList( + List.of( Batch.data( batch.baseOffset(), batch.partitionLeaderEpoch(), diff --git a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java index 30210fe0157..33a9333d475 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/AclControlManagerTest.java @@ -55,8 +55,6 @@ import org.junit.jupiter.api.Timeout; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -165,7 +163,7 @@ public class AclControlManagerTest { } static class MockClusterMetadataAuthorizer implements ClusterMetadataAuthorizer { - Map acls = Collections.emptyMap(); + Map acls = Map.of(); @Override public void setAclMutator(AclMutator aclMutator) { @@ -312,7 +310,7 @@ public class AclControlManagerTest { assertFalse(manager.idToAcl().isEmpty()); ControllerResult> deleteResult = - manager.deleteAcls(Arrays.asList( + manager.deleteAcls(List.of( new AclBindingFilter( new ResourcePatternFilter(ResourceType.ANY, null, LITERAL), AccessControlEntryFilter.ANY), @@ -326,7 +324,7 @@ public class AclControlManagerTest { assertEquals(Optional.empty(), result.exception()); deleted.add(result.aclBinding()); } - assertEquals(new HashSet<>(Arrays.asList( + assertEquals(new HashSet<>(List.of( TEST_ACLS.get(0).toBinding(), TEST_ACLS.get(2).toBinding())), deleted); assertEquals(InvalidRequestException.class, @@ -347,21 +345,21 @@ public class AclControlManagerTest { AclBinding aclBinding = new AclBinding(new ResourcePattern(TOPIC, "topic-1", LITERAL), new AccessControlEntry("User:user", "10.0.0.1", AclOperation.ALL, ALLOW)); - ControllerResult> createResult = manager.createAcls(Collections.singletonList(aclBinding)); + ControllerResult> createResult = manager.createAcls(List.of(aclBinding)); Uuid id = ((AccessControlEntryRecord) createResult.records().get(0).message()).id(); assertEquals(1, createResult.records().size()); - ControllerResult> deleteAclResultsAnyFilter = manager.deleteAcls(Collections.singletonList(AclBindingFilter.ANY)); + ControllerResult> deleteAclResultsAnyFilter = manager.deleteAcls(List.of(AclBindingFilter.ANY)); assertEquals(1, deleteAclResultsAnyFilter.records().size()); assertEquals(id, ((RemoveAccessControlEntryRecord) deleteAclResultsAnyFilter.records().get(0).message()).id()); assertEquals(1, deleteAclResultsAnyFilter.response().size()); - ControllerResult> deleteAclResultsSpecificFilter = manager.deleteAcls(Collections.singletonList(aclBinding.toFilter())); + ControllerResult> deleteAclResultsSpecificFilter = manager.deleteAcls(List.of(aclBinding.toFilter())); assertEquals(1, deleteAclResultsSpecificFilter.records().size()); assertEquals(id, ((RemoveAccessControlEntryRecord) deleteAclResultsSpecificFilter.records().get(0).message()).id()); assertEquals(1, deleteAclResultsSpecificFilter.response().size()); - ControllerResult> deleteAclResultsBothFilters = manager.deleteAcls(Arrays.asList(AclBindingFilter.ANY, aclBinding.toFilter())); + ControllerResult> deleteAclResultsBothFilters = manager.deleteAcls(List.of(AclBindingFilter.ANY, aclBinding.toFilter())); assertEquals(1, deleteAclResultsBothFilters.records().size()); assertEquals(id, ((RemoveAccessControlEntryRecord) deleteAclResultsBothFilters.records().get(0).message()).id()); assertEquals(2, deleteAclResultsBothFilters.response().size()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java index e6bb0940916..21bca1150fc 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BrokerHeartbeatManagerTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.metadata.placement.UsableBroker; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; import java.util.HashSet; import java.util.Iterator; import java.util.Optional; @@ -121,7 +120,7 @@ public class BrokerHeartbeatManagerTest { @Test public void testUsableBrokerIterator() { BrokerHeartbeatManager manager = newBrokerHeartbeatManager(); - assertEquals(Collections.emptySet(), usableBrokersToSet(manager)); + assertEquals(Set.of(), usableBrokersToSet(manager)); for (int brokerId = 0; brokerId < 5; brokerId++) { manager.register(brokerId, true); } @@ -152,7 +151,7 @@ public class BrokerHeartbeatManagerTest { @Test public void testControlledShutdownOffsetIsOnlyUpdatedOnce() { BrokerHeartbeatManager manager = newBrokerHeartbeatManager(); - assertEquals(Collections.emptySet(), usableBrokersToSet(manager)); + assertEquals(Set.of(), usableBrokersToSet(manager)); for (int brokerId = 0; brokerId < 5; brokerId++) { manager.register(brokerId, true); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokerToElrsTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokerToElrsTest.java index 32571bf081b..e3a96a2491c 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BrokerToElrsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BrokerToElrsTest.java @@ -24,8 +24,8 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.HashSet; +import java.util.List; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -37,7 +37,7 @@ public class BrokerToElrsTest { }; private static Set toSet(TopicIdPartition... partitions) { - return new HashSet<>(Arrays.asList(partitions)); + return new HashSet<>(List.of(partitions)); } private static Set toSet(BrokersToIsrs.PartitionsOnReplicaIterator iterator) { diff --git a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java index 391d301a00d..fc3b8d2899e 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/BrokersToIsrsTest.java @@ -26,8 +26,8 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.HashSet; +import java.util.List; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -41,7 +41,7 @@ public class BrokersToIsrsTest { }; private static Set toSet(TopicIdPartition... partitions) { - return new HashSet<>(Arrays.asList(partitions)); + return new HashSet<>(List.of(partitions)); } private static Set toSet(PartitionsOnReplicaIterator iterator) { diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java index 0c7116e926d..7532d2a6256 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClientQuotaControlManagerTest.java @@ -68,7 +68,7 @@ public class ClientQuotaControlManagerTest { assertInvalidEntity(manager, entity(ClientQuotaEntity.CLIENT_ID, "user-1", ClientQuotaEntity.IP, "1.2.3.4")); // Empty - assertInvalidEntity(manager, new ClientQuotaEntity(Collections.emptyMap())); + assertInvalidEntity(manager, new ClientQuotaEntity(Map.of())); } private void assertInvalidEntity(ClientQuotaControlManager manager, ClientQuotaEntity entity) { @@ -323,8 +323,8 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForEntityTypeWithUser() { - testConfigKeysForEntityType(Collections.singletonList(ClientQuotaEntity.USER), - Arrays.asList( + testConfigKeysForEntityType(List.of(ClientQuotaEntity.USER), + List.of( "producer_byte_rate", "consumer_byte_rate", "controller_mutation_rate", @@ -334,8 +334,8 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForEntityTypeWithClientId() { - testConfigKeysForEntityType(Collections.singletonList(ClientQuotaEntity.CLIENT_ID), - Arrays.asList( + testConfigKeysForEntityType(List.of(ClientQuotaEntity.CLIENT_ID), + List.of( "producer_byte_rate", "consumer_byte_rate", "controller_mutation_rate", @@ -345,8 +345,8 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForEntityTypeWithUserAndClientId() { - testConfigKeysForEntityType(Arrays.asList(ClientQuotaEntity.CLIENT_ID, ClientQuotaEntity.USER), - Arrays.asList( + testConfigKeysForEntityType(List.of(ClientQuotaEntity.CLIENT_ID, ClientQuotaEntity.USER), + List.of( "producer_byte_rate", "consumer_byte_rate", "controller_mutation_rate", @@ -356,8 +356,8 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForEntityTypeWithIp() { - testConfigKeysForEntityType(Collections.singletonList(ClientQuotaEntity.IP), - Collections.singletonList( + testConfigKeysForEntityType(List.of(ClientQuotaEntity.IP), + List.of( "connection_creation_rate" )); } @@ -386,20 +386,20 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForEmptyEntity() { - testConfigKeysError(Collections.emptyList(), + testConfigKeysError(List.of(), new ApiError(Errors.INVALID_REQUEST, "Invalid empty client quota entity")); } @Test public void testConfigKeysForEntityTypeWithIpAndUser() { - testConfigKeysError(Arrays.asList(ClientQuotaEntity.IP, ClientQuotaEntity.USER), + testConfigKeysError(List.of(ClientQuotaEntity.IP, ClientQuotaEntity.USER), new ApiError(Errors.INVALID_REQUEST, "Invalid quota entity combination, IP entity should" + "not be combined with User or ClientId")); } @Test public void testConfigKeysForEntityTypeWithIpAndClientId() { - testConfigKeysError(Arrays.asList(ClientQuotaEntity.IP, ClientQuotaEntity.CLIENT_ID), + testConfigKeysError(List.of(ClientQuotaEntity.IP, ClientQuotaEntity.CLIENT_ID), new ApiError(Errors.INVALID_REQUEST, "Invalid quota entity combination, IP entity should" + "not be combined with User or ClientId")); } @@ -410,7 +410,7 @@ public class ClientQuotaControlManagerTest { @Test public void testConfigKeysForUnresolvableIpEntity() { - testConfigKeysError(Collections.singletonMap(ClientQuotaEntity.IP, "example.invalid"), + testConfigKeysError(Map.of(ClientQuotaEntity.IP, "example.invalid"), new ApiError(Errors.INVALID_REQUEST, "example.invalid is not a valid IP or resolvable host.")); } @@ -427,7 +427,7 @@ public class ClientQuotaControlManagerTest { static { VALID_CLIENT_ID_QUOTA_KEYS = new HashMap<>(); assertEquals(ApiError.NONE, ClientQuotaControlManager.configKeysForEntityType( - keysToEntity(Collections.singletonList(ClientQuotaEntity.CLIENT_ID)), VALID_CLIENT_ID_QUOTA_KEYS)); + keysToEntity(List.of(ClientQuotaEntity.CLIENT_ID)), VALID_CLIENT_ID_QUOTA_KEYS)); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java index 5ca71c042c1..91e8d1a77f3 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ClusterControlManagerTest.java @@ -62,8 +62,6 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -71,9 +69,9 @@ import java.util.List; import java.util.Map; import java.util.Optional; import java.util.OptionalLong; +import java.util.Set; import java.util.stream.Stream; -import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -93,7 +91,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setTime(time). @@ -142,7 +140,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). @@ -195,7 +193,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). @@ -250,7 +248,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). @@ -268,7 +266,7 @@ public class ClusterControlManagerTest { setRack(null). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")), 123L, - new FinalizedControllerFeatures(Collections.emptyMap(), 456L), + new FinalizedControllerFeatures(Map.of(), 456L), false)); } @@ -288,7 +286,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -303,10 +301,10 @@ public class ClusterControlManagerTest { build(); clusterControl.activate(); - List logDirs = metadataVersion.isDirectoryAssignmentSupported() ? asList( + List logDirs = metadataVersion.isDirectoryAssignmentSupported() ? List.of( Uuid.fromString("63k9SN1nQOS0dFHSCIMA0A"), Uuid.fromString("Vm1MjsOCR1OjDDydOsDbzg") - ) : Collections.emptyList(); + ) : List.of(); ControllerResult result = clusterControl.registerBroker( new BrokerRegistrationRequestData(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). @@ -314,7 +312,7 @@ public class ClusterControlManagerTest { setLogDirs(logDirs). setRack(null). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(metadataVersion.featureLevel()). setMaxSupportedVersion(metadataVersion.featureLevel())).iterator())). @@ -326,14 +324,14 @@ public class ClusterControlManagerTest { short expectedVersion = metadataVersion.registerBrokerRecordVersion(); assertEquals( - Collections.singletonList(new ApiMessageAndVersion(new RegisterBrokerRecord(). + List.of(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(123L). setBrokerId(0). setRack(null). setIncarnationId(Uuid.fromString("0H4fUu1xQEKXFYwB1aBjhg")). setFenced(true). setLogDirs(logDirs). - setFeatures(new RegisterBrokerRecord.BrokerFeatureCollection(Collections.singletonList( + setFeatures(new RegisterBrokerRecord.BrokerFeatureCollection(List.of( new RegisterBrokerRecord.BrokerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(metadataVersion.featureLevel()). @@ -359,7 +357,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setTime(new MockTime(0, 0, 0)). @@ -374,7 +372,7 @@ public class ClusterControlManagerTest { setId(1). setEpoch(100). setIncarnationId(Uuid.fromString("fPZv1VBsRFmnlRvmGcOW9w")). - setListeners(Collections.singletonMap("PLAINTEXT", + setListeners(Map.of("PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 9092))). setRack(Optional.of("arack")). setFenced(true). @@ -398,7 +396,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setTime(time). @@ -461,7 +459,7 @@ public class ClusterControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -504,7 +502,7 @@ public class ClusterControlManagerTest { build(); assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(0).setRack(null). - setEndPoints(new BrokerEndpointCollection(Collections.singleton( + setEndPoints(new BrokerEndpointCollection(Set.of( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9092). setName("PLAINTEXT"). @@ -514,7 +512,7 @@ public class ClusterControlManagerTest { clusterControl.brokerRegistrations().get(0).toRecord(options)); assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(1).setRack(null). - setEndPoints(new BrokerEndpointCollection(Collections.singleton( + setEndPoints(new BrokerEndpointCollection(Set.of( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9093). setName("PLAINTEXT"). @@ -523,7 +521,7 @@ public class ClusterControlManagerTest { clusterControl.brokerRegistrations().get(1).toRecord(options)); assertEquals(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerEpoch(100).setBrokerId(2).setRack(null). - setEndPoints(new BrokerEndpointCollection(Collections.singleton( + setEndPoints(new BrokerEndpointCollection(Set.of( new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id). setPort((short) 9094). setName("PLAINTEXT"). @@ -544,7 +542,7 @@ public class ClusterControlManagerTest { TestFeatureVersion.TEST_1.featureLevel())); FeatureControlManager featureControl = new FeatureControlManager.Builder(). setSnapshotRegistry(snapshotRegistry). - setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, Collections.singletonList(0))). + setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, List.of(0))). build(); featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -561,7 +559,7 @@ public class ClusterControlManagerTest { setName(TestFeatureVersion.FEATURE_NAME).setFeatureLevel((short) 1); featureControl.replay(testFeatureRecord); - List logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); + List logDirs = List.of(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); BrokerRegistrationRequestData baseRequest = new BrokerRegistrationRequestData(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setBrokerId(0). @@ -574,7 +572,7 @@ public class ClusterControlManagerTest { assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_7_IV0.featureLevel())).iterator())), @@ -595,7 +593,7 @@ public class ClusterControlManagerTest { KRaftVersion.KRAFT_VERSION_1.featureLevel())); FeatureControlManager featureControl = new FeatureControlManager.Builder(). setSnapshotRegistry(snapshotRegistry). - setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, Collections.singletonList(0))). + setQuorumFeatures(new QuorumFeatures(0, supportedFeatures, List.of(0))). build(); featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -609,7 +607,7 @@ public class ClusterControlManagerTest { build(); clusterControl.activate(); - List logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); + List logDirs = List.of(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ")); BrokerRegistrationRequestData baseRequest = new BrokerRegistrationRequestData(). setClusterId("fPZv1VBsRFmnlRvmGcOW9w"). setBrokerId(0). @@ -627,7 +625,7 @@ public class ClusterControlManagerTest { assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel())).iterator())), @@ -640,7 +638,7 @@ public class ClusterControlManagerTest { assertThrows(UnsupportedVersionException.class, () -> clusterControl.registerBroker( baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Arrays.asList( + List.of( new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). @@ -655,7 +653,7 @@ public class ClusterControlManagerTest { clusterControl.registerBroker( baseRequest.setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Arrays.asList( + List.of( new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.IBP_3_9_IV0.featureLevel()). @@ -675,10 +673,10 @@ public class ClusterControlManagerTest { FeatureControlManager featureControl = new FeatureControlManager.Builder(). setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, - Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of( + Map.of(MetadataVersion.FEATURE_NAME, VersionRange.of( MetadataVersion.IBP_3_5_IV0.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())), - Collections.singletonList(0))). + List.of(0))). build(); featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -701,7 +699,7 @@ public class ClusterControlManagerTest { setBrokerId(0). setRack(null). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). @@ -719,7 +717,7 @@ public class ClusterControlManagerTest { setBrokerId(0). setRack(null). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_4_IV0.featureLevel())).iterator())). @@ -754,7 +752,7 @@ public class ClusterControlManagerTest { setFeatureControlManager(createFeatureControlManager()). setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). build(); - RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(0).setLogDirs(asList( + RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(0).setLogDirs(List.of( Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ") )); @@ -763,20 +761,20 @@ public class ClusterControlManagerTest { clusterControl.activate(); assertDoesNotThrow(() -> - registerNewBrokerWithDirs(clusterControl, 0, asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))), + registerNewBrokerWithDirs(clusterControl, 0, List.of(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))), "it should be possible to re-register the same broker with the same directories" ); assertEquals("No directories specified in request", assertThrows(InvalidRegistrationException.class, () -> - registerNewBrokerWithDirs(clusterControl, 1, Collections.emptyList()) + registerNewBrokerWithDirs(clusterControl, 1, List.of()) ).getMessage()); assertEquals("Broker 0 is already registered with directory Mj3CW3OSRi29cFeNJlXuAQ", assertThrows(InvalidRegistrationException.class, () -> - registerNewBrokerWithDirs(clusterControl, 1, asList(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))) + registerNewBrokerWithDirs(clusterControl, 1, List.of(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))) ).getMessage()); assertEquals("Reserved directory ID in request", assertThrows(InvalidRegistrationException.class, () -> - registerNewBrokerWithDirs(clusterControl, 1, asList(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"), DirectoryId.UNASSIGNED)) + registerNewBrokerWithDirs(clusterControl, 1, List.of(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"), DirectoryId.UNASSIGNED)) ).getMessage()); assertEquals("Duplicate directory ID in request", assertThrows(InvalidRegistrationException.class, () -> - registerNewBrokerWithDirs(clusterControl, 1, asList(Uuid.fromString("aR6lssMrSeyXRf65hiUovQ"), Uuid.fromString("aR6lssMrSeyXRf65hiUovQ"))) + registerNewBrokerWithDirs(clusterControl, 1, List.of(Uuid.fromString("aR6lssMrSeyXRf65hiUovQ"), Uuid.fromString("aR6lssMrSeyXRf65hiUovQ"))) ).getMessage()); } @@ -785,7 +783,7 @@ public class ClusterControlManagerTest { .setClusterId(clusterControl.clusterId()) .setIncarnationId(new Uuid(brokerId, brokerId)) .setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())) @@ -804,7 +802,7 @@ public class ClusterControlManagerTest { setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). build(); clusterControl.activate(); - registerNewBrokerWithDirs(clusterControl, 1, asList(Uuid.fromString("dir1SEbpRuG1dcpTRGOvJw"), Uuid.fromString("dir2xaEwR2m3JHTiy7PWwA"))); + registerNewBrokerWithDirs(clusterControl, 1, List.of(Uuid.fromString("dir1SEbpRuG1dcpTRGOvJw"), Uuid.fromString("dir2xaEwR2m3JHTiy7PWwA"))); assertTrue(clusterControl.registration(1).hasOnlineDir(Uuid.fromString("dir1SEbpRuG1dcpTRGOvJw"))); assertTrue(clusterControl.hasOnlineDir(1, Uuid.fromString("dir1SEbpRuG1dcpTRGOvJw"))); assertTrue(clusterControl.hasOnlineDir(1, Uuid.fromString("dir2xaEwR2m3JHTiy7PWwA"))); @@ -823,11 +821,11 @@ public class ClusterControlManagerTest { setBrokerShutdownHandler((brokerId, isCleanShutdown, records) -> { }). build(); clusterControl.activate(); - RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(1).setLogDirs(Collections.emptyList()); + RegisterBrokerRecord brokerRecord = new RegisterBrokerRecord().setBrokerEpoch(100).setBrokerId(1).setLogDirs(List.of()); brokerRecord.endPoints().add(new BrokerEndpoint().setSecurityProtocol(SecurityProtocol.PLAINTEXT.id).setPort((short) 9092).setName("PLAINTEXT").setHost("127.0.0.1")); clusterControl.replay(brokerRecord, 100L); - registerNewBrokerWithDirs(clusterControl, 2, Collections.singletonList(Uuid.fromString("singleOnlineDirectoryA"))); - registerNewBrokerWithDirs(clusterControl, 3, asList(Uuid.fromString("s4fRmyNFSH6J0vI8AVA5ew"), Uuid.fromString("UbtxBcqYSnKUEMcnTyZFWw"))); + registerNewBrokerWithDirs(clusterControl, 2, List.of(Uuid.fromString("singleOnlineDirectoryA"))); + registerNewBrokerWithDirs(clusterControl, 3, List.of(Uuid.fromString("s4fRmyNFSH6J0vI8AVA5ew"), Uuid.fromString("UbtxBcqYSnKUEMcnTyZFWw"))); assertEquals(DirectoryId.MIGRATING, clusterControl.defaultDir(1)); assertEquals(Uuid.fromString("singleOnlineDirectoryA"), clusterControl.defaultDir(2)); assertEquals(DirectoryId.UNASSIGNED, clusterControl.defaultDir(3)); @@ -850,12 +848,12 @@ public class ClusterControlManagerTest { setBrokerId(1). setClusterId(clusterControl.clusterId()). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). - setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), + setLogDirs(List.of(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 100, finalizedFeatures, false). @@ -865,13 +863,13 @@ public class ClusterControlManagerTest { setBrokerId(1). setClusterId(clusterControl.clusterId()). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). setIncarnationId(newIncarnationId ? Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww") : Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). - setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), + setLogDirs(List.of(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 111, finalizedFeatures, false). @@ -910,11 +908,11 @@ public class ClusterControlManagerTest { setClusterId(clusterControl.clusterId()). setIncarnationId(Uuid.fromString("mISEfEFwQIuaD1gKCc5tzQ")). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). - setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), + setLogDirs(List.of(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 100, finalizedFeatures, true). @@ -932,11 +930,11 @@ public class ClusterControlManagerTest { setIncarnationId(Uuid.fromString("07OOcU7MQFeSmGAFPP2Zww")). setPreviousBrokerEpoch(isCleanShutdown ? 100 : 10). setFeatures(new BrokerRegistrationRequestData.FeatureCollection( - Collections.singleton(new BrokerRegistrationRequestData.Feature(). + Set.of(new BrokerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel())).iterator())). - setLogDirs(Arrays.asList(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), + setLogDirs(List.of(Uuid.fromString("Vv1gzkM2QpuE-PPrIc6XEw"))), 111, finalizedFeatures, true).records(); @@ -964,11 +962,11 @@ public class ClusterControlManagerTest { clusterControl.replay(new RegisterBrokerRecord(). setBrokerEpoch(100). setBrokerId(0). - setLogDirs(asList(Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))), 10002); + setLogDirs(List.of(Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"))), 10002); clusterControl.replay(new RegisterBrokerRecord(). setBrokerEpoch(123). setBrokerId(1). - setLogDirs(asList(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"))), 10005); + setLogDirs(List.of(Uuid.fromString("TyNK6XSSQJaJc2q9uflNHg"))), 10005); clusterControl.activate(); assertEquals(OptionalLong.of(1000L), clusterControl.heartbeatManager().tracker(). contactTime(new BrokerIdAndEpoch(0, 100))); diff --git a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java index 0d1e465ab90..cd2b3fa18c3 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ConfigurationControlManagerTest.java @@ -44,8 +44,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -56,7 +54,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import static java.util.Arrays.asList; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.APPEND; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.DELETE; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; @@ -95,12 +92,12 @@ public class ConfigurationControlManagerTest { public static final Map> SYNONYMS = new HashMap<>(); static { - SYNONYMS.put("abc", Collections.singletonList(new ConfigSynonym("foo.bar"))); - SYNONYMS.put("def", Collections.singletonList(new ConfigSynonym("baz"))); + SYNONYMS.put("abc", List.of(new ConfigSynonym("foo.bar"))); + SYNONYMS.put("def", List.of(new ConfigSynonym("baz"))); SYNONYMS.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, - Collections.singletonList(new ConfigSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG))); - SYNONYMS.put("quuux", Collections.singletonList(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS))); - SYNONYMS.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, Collections.singletonList(new ConfigSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG))); + List.of(new ConfigSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG))); + SYNONYMS.put("quuux", List.of(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS))); + SYNONYMS.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, List.of(new ConfigSynonym(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG))); } static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, SYNONYMS); @@ -137,16 +134,16 @@ public class ConfigurationControlManagerTest { ConfigurationControlManager manager = new ConfigurationControlManager.Builder(). setKafkaConfigSchema(SCHEMA). build(); - assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0)); + assertEquals(Map.of(), manager.getConfigs(BROKER0)); manager.replay(new ConfigRecord(). setResourceType(BROKER.id()).setResourceName("0"). setName("foo.bar").setValue("1,2")); - assertEquals(Collections.singletonMap("foo.bar", "1,2"), + assertEquals(Map.of("foo.bar", "1,2"), manager.getConfigs(BROKER0)); manager.replay(new ConfigRecord(). setResourceType(BROKER.id()).setResourceName("0"). setName("foo.bar").setValue(null)); - assertEquals(Collections.emptyMap(), manager.getConfigs(BROKER0)); + assertEquals(Map.of(), manager.getConfigs(BROKER0)); manager.replay(new ConfigRecord(). setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("x,y,z")); @@ -172,7 +169,7 @@ public class ConfigurationControlManagerTest { entry(MYTOPIC, toMap(entry("abc", entry(APPEND, "123"))))), true); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("123"), CONFIG_RECORD.highestSupportedVersion())), toMap(entry(BROKER0, new ApiError(Errors.INVALID_CONFIG, @@ -181,7 +178,7 @@ public class ConfigurationControlManagerTest { RecordTestUtils.replayAll(manager, result.records()); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue(null), CONFIG_RECORD.highestSupportedVersion())), toMap(entry(MYTOPIC, ApiError.NONE))), @@ -201,14 +198,14 @@ public class ConfigurationControlManagerTest { ControllerResult result = manager. incrementalAlterConfig(MYTOPIC, keyToOps, true); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("123"), CONFIG_RECORD.highestSupportedVersion())), ApiError.NONE), result); RecordTestUtils.replayAll(manager, result.records()); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue(null), CONFIG_RECORD.highestSupportedVersion())), ApiError.NONE), @@ -234,7 +231,7 @@ public class ConfigurationControlManagerTest { ControllerResult> result = manager. incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap(entry("abc", entry(APPEND, "123,456,789"))))), true); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("123,456,789"), CONFIG_RECORD.highestSupportedVersion())), toMap(entry(MYTOPIC, ApiError.NONE))), result); @@ -245,14 +242,14 @@ public class ConfigurationControlManagerTest { result = manager .incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap(entry("abc", entry(APPEND, "123,456"))))), true); assertEquals( - ControllerResult.atomicOf(Collections.emptyList(), toMap(entry(MYTOPIC, ApiError.NONE))), + ControllerResult.atomicOf(List.of(), toMap(entry(MYTOPIC, ApiError.NONE))), result ); RecordTestUtils.replayAll(manager, result.records()); result = manager .incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap(entry("abc", entry(SUBTRACT, "123,456"))))), true); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("789"), CONFIG_RECORD.highestSupportedVersion())), toMap(entry(MYTOPIC, ApiError.NONE))), @@ -263,7 +260,7 @@ public class ConfigurationControlManagerTest { result = manager .incrementalAlterConfigs(toMap(entry(MYTOPIC, toMap(entry("abc", entry(SUBTRACT, "123456"))))), true); assertEquals( - ControllerResult.atomicOf(Collections.emptyList(), toMap(entry(MYTOPIC, ApiError.NONE))), + ControllerResult.atomicOf(List.of(), toMap(entry(MYTOPIC, ApiError.NONE))), result ); RecordTestUtils.replayAll(manager, result.records()); @@ -286,7 +283,7 @@ public class ConfigurationControlManagerTest { entry(existingTopic, toMap(entry("def", entry(SET, "newVal"))))), false); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(TOPIC.id()).setResourceName("ExistingTopic"). setName("def").setValue("newVal"), CONFIG_RECORD.highestSupportedVersion())), toMap(entry(BROKER0, new ApiError(Errors.UNKNOWN_TOPIC_OR_PARTITION, @@ -329,8 +326,8 @@ public class ConfigurationControlManagerTest { @Test public void testIncrementalAlterConfigsWithPolicy() { - MockAlterConfigsPolicy policy = new MockAlterConfigsPolicy(asList( - new RequestMetadata(MYTOPIC, Collections.emptyMap()), + MockAlterConfigsPolicy policy = new MockAlterConfigsPolicy(List.of( + new RequestMetadata(MYTOPIC, Map.of()), new RequestMetadata(BROKER0, toMap( entry("foo.bar", "123"), entry("quux", "456"), @@ -347,7 +344,7 @@ public class ConfigurationControlManagerTest { setName("topic.config").setValue("123")); manager.replay(new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). setName("broker.config.to.remove").setValue("123")); - assertEquals(ControllerResult.atomicOf(asList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). setName("foo.bar").setValue("123"), CONFIG_RECORD.highestSupportedVersion()), new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("0"). @@ -399,7 +396,7 @@ public class ConfigurationControlManagerTest { setKafkaConfigSchema(SCHEMA). setAlterConfigPolicy(Optional.of(new CheckForNullValuesPolicy())). build(); - List expectedRecords1 = asList( + List expectedRecords1 = List.of( new ApiMessageAndVersion(new ConfigRecord(). setResourceType(TOPIC.id()).setResourceName("mytopic"). setName("abc").setValue("456"), CONFIG_RECORD.highestSupportedVersion()), @@ -414,7 +411,7 @@ public class ConfigurationControlManagerTest { for (ApiMessageAndVersion message : expectedRecords1) { manager.replay((ConfigRecord) message.message()); } - assertEquals(ControllerResult.atomicOf(Collections.singletonList( + assertEquals(ControllerResult.atomicOf(List.of( new ApiMessageAndVersion( new ConfigRecord() .setResourceType(TOPIC.id()) @@ -441,9 +438,9 @@ public class ConfigurationControlManagerTest { toMap(entry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, entry(SET, "3"))); ConfigResource brokerConfigResource = new ConfigResource(ConfigResource.Type.BROKER, "1"); ControllerResult result = manager.incrementalAlterConfig(brokerConfigResource, keyToOps, true); - assertEquals(Collections.emptySet(), manager.brokersWithConfigs()); + assertEquals(Set.of(), manager.brokersWithConfigs()); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new ConfigRecord().setResourceType(BROKER.id()).setResourceName("1"). setName(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG).setValue("3"), (short) 0)), ApiError.NONE), result); @@ -457,7 +454,7 @@ public class ConfigurationControlManagerTest { effectiveMinInsync + ". Removing broker-level min.insync.replicas " + "for brokers: 1.", manager.maybeGenerateElrSafetyRecords(records)); - assertEquals(Arrays.asList(new ApiMessageAndVersion( + assertEquals(List.of(new ApiMessageAndVersion( new ConfigRecord(). setResourceType(BROKER.id()). setResourceName(""). @@ -470,7 +467,7 @@ public class ConfigurationControlManagerTest { setValue(null), (short) 0)), records); RecordTestUtils.replayAll(manager, records); - assertEquals(Collections.emptySet(), manager.brokersWithConfigs()); + assertEquals(Set.of(), manager.brokersWithConfigs()); } @ParameterizedTest @@ -479,7 +476,7 @@ public class ConfigurationControlManagerTest { FeatureControlManager featureManager = new FeatureControlManager.Builder(). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.emptyList())). + List.of())). build(); featureManager.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -490,9 +487,9 @@ public class ConfigurationControlManagerTest { setKafkaConfigSchema(SCHEMA). build(); ControllerResult result = manager.updateFeatures( - Collections.singletonMap(EligibleLeaderReplicasVersion.FEATURE_NAME, + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()), - Collections.singletonMap(EligibleLeaderReplicasVersion.FEATURE_NAME, + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), false); assertNotNull(result.response()); @@ -529,7 +526,7 @@ public class ConfigurationControlManagerTest { FeatureControlManager featureManager = new FeatureControlManager.Builder(). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.emptyList())). + List.of())). build(); featureManager.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -541,9 +538,9 @@ public class ConfigurationControlManagerTest { build(); assertFalse(featureManager.isElrFeatureEnabled()); ControllerResult result = manager.updateFeatures( - Collections.singletonMap(EligibleLeaderReplicasVersion.FEATURE_NAME, + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()), - Collections.singletonMap(EligibleLeaderReplicasVersion.FEATURE_NAME, + Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), false); assertNotNull(result.response()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java index 52e06a0e505..55ec4749264 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/FeatureControlManagerTest.java @@ -40,8 +40,6 @@ import org.junit.jupiter.api.Timeout; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -49,8 +47,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import static java.util.Collections.emptyList; -import static java.util.Collections.singletonMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -83,7 +79,7 @@ public class FeatureControlManagerTest { public static QuorumFeatures features(Object... args) { Map features = QuorumFeatures.defaultSupportedFeatureMap(true); features.putAll(rangeMap(args)); - return new QuorumFeatures(0, features, emptyList()); + return new QuorumFeatures(0, features, List.of()); } private static Map updateMap(Object... args) { @@ -105,15 +101,15 @@ public class FeatureControlManagerTest { build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); snapshotRegistry.idempotentCreateSnapshot(-1); - assertEquals(new FinalizedControllerFeatures(Collections.singletonMap("metadata.version", MetadataVersion.MINIMUM_VERSION.featureLevel()), -1), + assertEquals(new FinalizedControllerFeatures(Map.of("metadata.version", MetadataVersion.MINIMUM_VERSION.featureLevel()), -1), manager.finalizedFeatures(-1)); - assertEquals(ControllerResult.of(emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 3 for feature " + TestFeatureVersion.FEATURE_NAME + ". Local controller 0 only supports versions 0-2")), manager.updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 3), - Collections.singletonMap(TestFeatureVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of(TestFeatureVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), false)); ControllerResult result = manager.updateFeatures( - updateMap(TestFeatureVersion.FEATURE_NAME, 1, "bar", 1), Collections.emptyMap(), + updateMap(TestFeatureVersion.FEATURE_NAME, 1, "bar", 1), Map.of(), false); ApiError expectedError = new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 1 for feature bar. Local controller 0 does not support this feature."); @@ -122,7 +118,7 @@ public class FeatureControlManagerTest { assertEquals(expectedMessages, result.records()); result = manager.updateFeatures( - updateMap(TestFeatureVersion.FEATURE_NAME, 1), Collections.emptyMap(), + updateMap(TestFeatureVersion.FEATURE_NAME, 1), Map.of(), false); expectedError = ApiError.NONE; assertEquals(expectedError, result.response()); @@ -203,31 +199,31 @@ public class FeatureControlManagerTest { setQuorumFeatures(features("foo", 1, 5, TransactionVersion.FEATURE_NAME, 0, 3)). setSnapshotRegistry(snapshotRegistry). setClusterFeatureSupportDescriber(createFakeClusterFeatureSupportDescriber( - Collections.singletonList(new SimpleImmutableEntry<>(5, Collections.singletonMap(TransactionVersion.FEATURE_NAME, VersionRange.of(0, 2)))), - emptyList())). + List.of(new SimpleImmutableEntry<>(5, Map.of(TransactionVersion.FEATURE_NAME, VersionRange.of(0, 2)))), + List.of())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); - assertEquals(ControllerResult.of(emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 3 for feature foo. Broker 5 does not support this feature.")), manager.updateFeatures(updateMap("foo", 3), - Collections.singletonMap("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of("foo", FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), false)); ControllerResult result = manager.updateFeatures( - updateMap(TransactionVersion.FEATURE_NAME, 2), Collections.emptyMap(), false); + updateMap(TransactionVersion.FEATURE_NAME, 2), Map.of(), false); assertEquals(ApiError.NONE, result.response()); manager.replay((FeatureLevelRecord) result.records().get(0).message()); snapshotRegistry.idempotentCreateSnapshot(3); - assertEquals(ControllerResult.of(emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 1 for feature " + TransactionVersion.FEATURE_NAME + "." + " Can't downgrade the version of this feature without setting the upgrade type to either safe or unsafe downgrade.")), - manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Collections.emptyMap(), false)); + manager.updateFeatures(updateMap(TransactionVersion.FEATURE_NAME, 1), Map.of(), false)); assertEquals( ControllerResult.atomicOf( - Collections.singletonList( + List.of( new ApiMessageAndVersion( new FeatureLevelRecord() .setName(TransactionVersion.FEATURE_NAME) @@ -239,7 +235,7 @@ public class FeatureControlManagerTest { ), manager.updateFeatures( updateMap(TransactionVersion.FEATURE_NAME, 1), - Collections.singletonMap(TransactionVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of(TransactionVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), false) ); } @@ -255,12 +251,12 @@ public class FeatureControlManagerTest { build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); ControllerResult result = manager. - updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Collections.emptyMap(), false); + updateFeatures(updateMap(TestFeatureVersion.FEATURE_NAME, 1, TransactionVersion.FEATURE_NAME, 2), Map.of(), false); RecordTestUtils.replayAll(manager, result.records()); assertEquals(MetadataVersion.MINIMUM_VERSION, manager.metadataVersionOrThrow()); assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(TestFeatureVersion.FEATURE_NAME)); assertEquals(Optional.of((short) 2), manager.finalizedFeatures(Long.MAX_VALUE).get(TransactionVersion.FEATURE_NAME)); - assertEquals(new HashSet<>(Arrays.asList( + assertEquals(new HashSet<>(List.of( MetadataVersion.FEATURE_NAME, TestFeatureVersion.FEATURE_NAME, TransactionVersion.FEATURE_NAME)), manager.finalizedFeatures(Long.MAX_VALUE).featureNames()); } @@ -287,24 +283,24 @@ public class FeatureControlManagerTest { @Test public void testCannotDowngradeToHigherVersion() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 9 for feature metadata.version. Can't downgrade to a " + "newer version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @Test public void testCannotUnsafeDowngradeToHigherVersion() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 9 for feature metadata.version. Can't downgrade to a " + "newer version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -315,46 +311,46 @@ public class FeatureControlManagerTest { MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.IBP_3_5_IV1.featureLevel())); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 9 for feature metadata.version. Can't downgrade the " + "version of this feature without setting the upgrade type to either safe or " + "unsafe downgrade.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), true)); } @Test public void testCanUpgradeToHigherVersion() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), + assertEquals(ControllerResult.of(List.of(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_5_IV0.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UPGRADE), true)); } @Test public void testCannotUseSafeDowngradeIfMetadataChanged() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid metadata.version 7. Refusing to perform the requested downgrade because " + "it might delete metadata information.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @Test public void testUnsafeDowngradeIsTemporarilyDisabled() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid metadata.version 7. Unsafe metadata downgrade is not supported in this version.")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -362,10 +358,10 @@ public class FeatureControlManagerTest { @Test public void testCanUseUnsafeDowngradeIfMetadataChanged() { FeatureControlManager manager = createTestManager(); - assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), + assertEquals(ControllerResult.of(List.of(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_3_IV3.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -376,10 +372,10 @@ public class FeatureControlManagerTest { MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.IBP_3_5_IV0.featureLevel())); - assertEquals(ControllerResult.of(Collections.emptyList(), ApiError.NONE), + assertEquals(ControllerResult.of(List.of(), ApiError.NONE), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersion.IBP_3_4_IV0.featureLevel()), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.SAFE_DOWNGRADE), true)); } @@ -390,11 +386,11 @@ public class FeatureControlManagerTest { MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); - assertEquals(ControllerResult.of(Collections.emptyList(), new ApiError(Errors.INVALID_UPDATE_VERSION, + assertEquals(ControllerResult.of(List.of(), new ApiError(Errors.INVALID_UPDATE_VERSION, "Invalid update version 6 for feature metadata.version. Local controller 0 only supports versions 7-26")), manager.updateFeatures( - singletonMap(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), - singletonMap(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + Map.of(MetadataVersion.FEATURE_NAME, MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL), + Map.of(MetadataVersion.FEATURE_NAME, FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), true)); } @@ -405,27 +401,27 @@ public class FeatureControlManagerTest { MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.latestTesting().featureLevel())); localSupportedFeatures.put(Feature.TEST_VERSION.featureName(), VersionRange.of(0, 2)); FeatureControlManager manager = new FeatureControlManager.Builder(). - setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())). + setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, List.of())). setClusterFeatureSupportDescriber(createFakeClusterFeatureSupportDescriber( - Collections.singletonList(new SimpleImmutableEntry<>(1, Collections.singletonMap(Feature.TEST_VERSION.featureName(), VersionRange.of(0, 3)))), - emptyList())). + List.of(new SimpleImmutableEntry<>(1, Map.of(Feature.TEST_VERSION.featureName(), VersionRange.of(0, 3)))), + List.of())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); ControllerResult result = manager.updateFeatures( - Collections.singletonMap(Feature.TEST_VERSION.featureName(), (short) 1), - Collections.singletonMap(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), + Map.of(Feature.TEST_VERSION.featureName(), (short) 1), + Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), false); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 1), (short) 0)), ApiError.NONE), result); RecordTestUtils.replayAll(manager, result.records()); assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(Feature.TEST_VERSION.featureName())); ControllerResult result2 = manager.updateFeatures( - Collections.singletonMap(Feature.TEST_VERSION.featureName(), (short) 0), - Collections.singletonMap(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), + Map.of(Feature.TEST_VERSION.featureName(), (short) 0), + Map.of(Feature.TEST_VERSION.featureName(), FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE), false); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new FeatureLevelRecord().setName(Feature.TEST_VERSION.featureName()).setFeatureLevel((short) 0), (short) 0)), ApiError.NONE), result2); RecordTestUtils.replayAll(manager, result2.records()); @@ -439,18 +435,18 @@ public class FeatureControlManagerTest { MetadataVersion.IBP_4_0_IV1.featureLevel(), MetadataVersion.latestTesting().featureLevel())); localSupportedFeatures.put(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), VersionRange.of(0, 1)); FeatureControlManager manager = new FeatureControlManager.Builder(). - setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, emptyList())). + setQuorumFeatures(new QuorumFeatures(0, localSupportedFeatures, List.of())). setClusterFeatureSupportDescriber(createFakeClusterFeatureSupportDescriber( - Collections.singletonList(new SimpleImmutableEntry<>(1, Collections.singletonMap(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), VersionRange.of(0, 1)))), - emptyList())). + List.of(new SimpleImmutableEntry<>(1, Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), VersionRange.of(0, 1)))), + List.of())). build(); manager.replay(new FeatureLevelRecord().setName(MetadataVersion.FEATURE_NAME).setFeatureLevel(MetadataVersion.IBP_4_0_IV1.featureLevel())); ControllerResult result = manager.updateFeatures( - Collections.singletonMap(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), (short) 1), - Collections.singletonMap(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), + Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), (short) 1), + Map.of(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName(), FeatureUpdate.UpgradeType.UPGRADE), false); assertTrue(result.response().isSuccess()); - assertEquals(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(List.of(new ApiMessageAndVersion( new FeatureLevelRecord(). setName(Feature.ELIGIBLE_LEADER_REPLICAS_VERSION.featureName()). setFeatureLevel((short) 1), (short) 0)), diff --git a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java index 8b646f4eac6..57f0df436c1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/OffsetControlManagerTest.java @@ -32,8 +32,7 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -55,7 +54,7 @@ public class OffsetControlManagerTest { assertEquals(-1L, offsetControl.transactionStartOffset()); assertEquals(-1L, offsetControl.nextWriteOffset()); assertFalse(offsetControl.active()); - assertEquals(Collections.singletonList(-1L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(-1L), offsetControl.snapshotRegistry().epochsList()); } @Test @@ -65,7 +64,7 @@ public class OffsetControlManagerTest { assertEquals(1000L, offsetControl.nextWriteOffset()); assertTrue(offsetControl.active()); assertTrue(offsetControl.metrics().active()); - assertEquals(Collections.singletonList(-1L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(-1L), offsetControl.snapshotRegistry().epochsList()); } @Test @@ -113,7 +112,7 @@ public class OffsetControlManagerTest { epoch, appendTimestamp, 100, - Collections.singletonList(new ApiMessageAndVersion(new NoOpRecord(), (short) 0))); + List.of(new ApiMessageAndVersion(new NoOpRecord(), (short) 0))); } @Test @@ -121,7 +120,7 @@ public class OffsetControlManagerTest { OffsetControlManager offsetControl = new OffsetControlManager.Builder().build(); offsetControl.handleCommitBatch(newFakeBatch(1000L, 200, 3000L)); - assertEquals(Collections.singletonList(1000L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(1000L), offsetControl.snapshotRegistry().epochsList()); assertEquals(1000L, offsetControl.lastCommittedOffset()); assertEquals(200, offsetControl.lastCommittedEpoch()); assertEquals(1000L, offsetControl.lastStableOffset()); @@ -143,12 +142,12 @@ public class OffsetControlManagerTest { assertEquals(2000L, offsetControl.metrics().lastAppliedRecordOffset()); assertEquals(-1L, offsetControl.lastStableOffset()); assertEquals(-1L, offsetControl.lastCommittedOffset()); - assertEquals(Arrays.asList(-1L, 2000L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(-1L, 2000L), offsetControl.snapshotRegistry().epochsList()); offsetControl.handleCommitBatch(newFakeBatch(2000L, 200, 3000L)); assertEquals(2000L, offsetControl.lastStableOffset()); assertEquals(2000L, offsetControl.lastCommittedOffset()); - assertEquals(Collections.singletonList(2000L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(2000L), offsetControl.snapshotRegistry().epochsList()); } @Test @@ -159,17 +158,17 @@ public class OffsetControlManagerTest { build(); offsetControl.beginLoadSnapshot(new OffsetAndEpoch(4000L, 300)); - assertEquals(Arrays.asList("snapshot[-1]", "reset"), snapshotRegistry.operations()); + assertEquals(List.of("snapshot[-1]", "reset"), snapshotRegistry.operations()); assertEquals(new OffsetAndEpoch(4000L, 300), offsetControl.currentSnapshotId()); assertEquals("00000000000000004000-0000000300", offsetControl.currentSnapshotName()); - assertEquals(Collections.emptyList(), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(), offsetControl.snapshotRegistry().epochsList()); offsetControl.endLoadSnapshot(3456L); - assertEquals(Arrays.asList("snapshot[-1]", "reset", "snapshot[4000]"), + assertEquals(List.of("snapshot[-1]", "reset", "snapshot[4000]"), snapshotRegistry.operations()); assertNull(offsetControl.currentSnapshotId()); assertNull(offsetControl.currentSnapshotName()); - assertEquals(Collections.singletonList(4000L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(4000L), offsetControl.snapshotRegistry().epochsList()); assertEquals(4000L, offsetControl.lastCommittedOffset()); assertEquals(300, offsetControl.lastCommittedEpoch()); assertEquals(4000L, offsetControl.lastStableOffset()); @@ -229,21 +228,21 @@ public class OffsetControlManagerTest { offsetControl.replay(new BeginTransactionRecord(), 1500L); assertEquals(1500L, offsetControl.transactionStartOffset()); - assertEquals(Arrays.asList(-1L, 1499L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(-1L, 1499L), offsetControl.snapshotRegistry().epochsList()); offsetControl.handleCommitBatch(newFakeBatch(1550L, 100, 2000L)); assertEquals(1550L, offsetControl.lastCommittedOffset()); assertEquals(100, offsetControl.lastCommittedEpoch()); assertEquals(1499L, offsetControl.lastStableOffset()); - assertEquals(Collections.singletonList(1499L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(1499L), offsetControl.snapshotRegistry().epochsList()); if (aborted) { offsetControl.replay(new AbortTransactionRecord(), 1600L); - assertEquals(Arrays.asList("snapshot[-1]", "snapshot[1499]", "revert[1499]"), + assertEquals(List.of("snapshot[-1]", "snapshot[1499]", "revert[1499]"), snapshotRegistry.operations()); } else { offsetControl.replay(new EndTransactionRecord(), 1600L); - assertEquals(Arrays.asList("snapshot[-1]", "snapshot[1499]"), + assertEquals(List.of("snapshot[-1]", "snapshot[1499]"), snapshotRegistry.operations()); } assertEquals(-1L, offsetControl.transactionStartOffset()); @@ -251,7 +250,7 @@ public class OffsetControlManagerTest { offsetControl.handleCommitBatch(newFakeBatch(1650, 100, 2100L)); assertEquals(1650, offsetControl.lastStableOffset()); - assertEquals(Collections.singletonList(1650L), offsetControl.snapshotRegistry().epochsList()); + assertEquals(List.of(1650L), offsetControl.snapshotRegistry().epochsList()); } @Test @@ -263,7 +262,7 @@ public class OffsetControlManagerTest { offsetControl.replay(new BeginTransactionRecord(), 1500L); offsetControl.beginLoadSnapshot(new OffsetAndEpoch(4000L, 300)); assertEquals(-1L, offsetControl.transactionStartOffset()); - assertEquals(Arrays.asList("snapshot[-1]", "snapshot[1499]", "reset"), + assertEquals(List.of("snapshot[-1]", "snapshot[1499]", "reset"), snapshotRegistry.operations()); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java index 2cca336bd69..bd7de9bbd15 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionChangeBuilderTest.java @@ -37,8 +37,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.function.IntPredicate; @@ -80,22 +79,22 @@ public class PartitionChangeBuilderTest { assertTrue(changeRecordIsNoOp(new PartitionChangeRecord())); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord().setLeader(1))); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord(). - setIsr(Arrays.asList(1, 2, 3)))); + setIsr(List.of(1, 2, 3)))); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord(). - setRemovingReplicas(Collections.singletonList(1)))); + setRemovingReplicas(List.of(1)))); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord(). - setAddingReplicas(Collections.singletonList(4)))); + setAddingReplicas(List.of(4)))); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord(). - setEligibleLeaderReplicas(Collections.singletonList(5)))); + setEligibleLeaderReplicas(List.of(5)))); assertFalse(changeRecordIsNoOp(new PartitionChangeRecord(). - setLastKnownElr(Collections.singletonList(6)))); + setLastKnownElr(List.of(6)))); assertFalse( changeRecordIsNoOp( new PartitionChangeRecord() .setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()) ) ); - assertFalse(changeRecordIsNoOp(new PartitionChangeRecord().setDirectories(Arrays.asList( + assertFalse(changeRecordIsNoOp(new PartitionChangeRecord().setDirectories(List.of( Uuid.fromString("5JwD0VNXRV2Wr9CCON38Tw"), Uuid.fromString("zpL1bRzTQXmmgdxlLHOWuw"), Uuid.fromString("6iGUpAkHQXC6bY0FTcPRDw") @@ -270,16 +269,16 @@ public class PartitionChangeBuilderTest { assertElectLeaderEquals(createFooBuilder(version), 1, false); assertElectLeaderEquals(createFooBuilder(version).setElection(Election.UNCLEAN), 1, false); assertElectLeaderEquals(createFooBuilder(version) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 3))), 1, false); + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 3))), 1, false); assertElectLeaderEquals(createFooBuilder(version).setElection(Election.UNCLEAN) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 3))), 1, false); + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 3))), 1, false); assertElectLeaderEquals(createFooBuilder(version) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Collections.singletonList(3))), NO_LEADER, false); + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(3))), NO_LEADER, false); assertElectLeaderEquals(createFooBuilder(version).setElection(Election.UNCLEAN). - setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Collections.singletonList(3))), 2, true); + setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(3))), 2, true); assertElectLeaderEquals( createFooBuilder(version).setElection(Election.UNCLEAN) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Collections.singletonList(4))).setTargetReplicas(Arrays.asList(2, 1, 3, 4)), + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(4))).setTargetReplicas(List.of(2, 1, 3, 4)), 4, false ); @@ -317,7 +316,7 @@ public class PartitionChangeBuilderTest { MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1))), + AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1))), new PartitionChangeRecord(), NO_LEADER_CHANGE); } @@ -332,7 +331,7 @@ public class PartitionChangeBuilderTest { MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1))), + AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1))), new PartitionChangeRecord(), 1); } @@ -346,7 +345,7 @@ public class PartitionChangeBuilderTest { MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); testTriggerLeaderEpochBumpIfNeeded( createFooBuilder(metadataVersion).setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1, 3, 4))), + AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1, 3, 4))), new PartitionChangeRecord(), NO_LEADER_CHANGE); } @@ -360,7 +359,7 @@ public class PartitionChangeBuilderTest { public void testLeaderEpochBumpOnNewReplicaSetDisjoint(String metadataVersionString) { MetadataVersion metadataVersion = MetadataVersion.fromVersionString(metadataVersionString); testTriggerLeaderEpochBumpIfNeeded( - createFooBuilder(metadataVersion).setTargetReplicas(Arrays.asList(2, 1, 4)), + createFooBuilder(metadataVersion).setTargetReplicas(List.of(2, 1, 4)), new PartitionChangeRecord(), 1); } @@ -392,7 +391,7 @@ public class PartitionChangeBuilderTest { 2). setEligibleLeaderReplicasEnabled(metadataVersion.isElrSupported()). setDefaultDirProvider(DEFAULT_DIR_PROVIDER). - setTargetReplicas(Collections.emptyList()); + setTargetReplicas(List.of()); PartitionChangeRecord record = new PartitionChangeRecord(); builder.triggerLeaderEpochBumpForIsrShrinkIfNeeded(record); assertEquals(NO_LEADER_CHANGE, record.leader()); @@ -419,14 +418,14 @@ public class PartitionChangeBuilderTest { new PartitionChangeRecord() .setTopicId(FOO_ID) .setPartitionId(0) - .setIsr(Arrays.asList(2, 1)), + .setIsr(List.of(2, 1)), version ) ), // Actual createFooBuilder(version) .setTargetIsrWithBrokerStates( - AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 1)) + AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 1)) ) .build() ); @@ -438,10 +437,10 @@ public class PartitionChangeBuilderTest { assertEquals(Optional.of(new ApiMessageAndVersion(new PartitionChangeRecord(). setTopicId(FOO_ID). setPartitionId(0). - setIsr(Arrays.asList(2, 3)). + setIsr(List.of(2, 3)). setLeader(2), version)), createFooBuilder(version).setTargetIsrWithBrokerStates(AlterPartitionRequest. - newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(2, 3))).build()); + newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2, 3))).build()); } @ParameterizedTest @@ -450,13 +449,13 @@ public class PartitionChangeBuilderTest { PartitionChangeRecord expectedRecord = new PartitionChangeRecord(). setTopicId(FOO_ID). setPartitionId(0). - setReplicas(Arrays.asList(3, 2, 1)); + setReplicas(List.of(3, 2, 1)); if (version >= 1) { Map dirs = DirectoryId.createAssignmentMap(FOO.replicas, FOO.directories); - expectedRecord.setDirectories(Arrays.asList(dirs.get(3), dirs.get(2), dirs.get(1))); + expectedRecord.setDirectories(List.of(dirs.get(3), dirs.get(2), dirs.get(1))); } assertEquals(Optional.of(new ApiMessageAndVersion(expectedRecord, version)), - createFooBuilder(version).setTargetReplicas(Arrays.asList(3, 2, 1)).build()); + createFooBuilder(version).setTargetReplicas(List.of(3, 2, 1)).build()); } @ParameterizedTest @@ -465,43 +464,43 @@ public class PartitionChangeBuilderTest { PartitionChangeRecord expectedRecord = new PartitionChangeRecord(). setTopicId(BAR_ID). setPartitionId(0). - setReplicas(Arrays.asList(2, 3, 4)). - setIsr(Arrays.asList(2, 3, 4)). + setReplicas(List.of(2, 3, 4)). + setIsr(List.of(2, 3, 4)). setLeader(2). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()); + setRemovingReplicas(List.of()). + setAddingReplicas(List.of()); if (version >= 1) { Map dirs = DirectoryId.createAssignmentMap(BAR.replicas, BAR.directories); - expectedRecord.setDirectories(Arrays.asList(dirs.get(2), dirs.get(3), dirs.get(4))); + expectedRecord.setDirectories(List.of(dirs.get(2), dirs.get(3), dirs.get(4))); } assertEquals(Optional.of(new ApiMessageAndVersion(expectedRecord, version)), createBarBuilder(version).setTargetIsrWithBrokerStates(AlterPartitionRequest. - newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3, 4))).build()); + newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 2, 3, 4))).build()); } @ParameterizedTest @MethodSource("partitionChangeRecordVersions") public void testRevertReassignment(short version) { PartitionReassignmentRevert revert = new PartitionReassignmentRevert(BAR); - assertEquals(Arrays.asList(1, 2, 3), revert.replicas()); - assertEquals(Arrays.asList(1, 2, 3), revert.isr()); + assertEquals(List.of(1, 2, 3), revert.replicas()); + assertEquals(List.of(1, 2, 3), revert.isr()); PartitionChangeRecord expectedRecord = new PartitionChangeRecord(). setTopicId(BAR_ID). setPartitionId(0). - setReplicas(Arrays.asList(1, 2, 3)). + setReplicas(List.of(1, 2, 3)). setLeader(1). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()); + setRemovingReplicas(List.of()). + setAddingReplicas(List.of()); if (version >= 1) { Map dirs = DirectoryId.createAssignmentMap(BAR.replicas, BAR.directories); - expectedRecord.setDirectories(Arrays.asList(dirs.get(1), dirs.get(2), dirs.get(3))); + expectedRecord.setDirectories(List.of(dirs.get(1), dirs.get(2), dirs.get(3))); } assertEquals(Optional.of(new ApiMessageAndVersion(expectedRecord, version)), createBarBuilder(version). setTargetReplicas(revert.replicas()). setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(revert.isr())). - setTargetRemoving(Collections.emptyList()). - setTargetAdding(Collections.emptyList()). + setTargetRemoving(List.of()). + setTargetAdding(List.of()). build()); } @@ -509,19 +508,19 @@ public class PartitionChangeBuilderTest { @MethodSource("partitionChangeRecordVersions") public void testRemovingReplicaReassignment(short version) { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2))); - assertEquals(Collections.singletonList(3), replicas.removing()); - assertEquals(Collections.emptyList(), replicas.adding()); - assertEquals(Arrays.asList(1, 2, 3), replicas.replicas()); + partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(List.of(1, 2))); + assertEquals(List.of(3), replicas.removing()); + assertEquals(List.of(), replicas.adding()); + assertEquals(List.of(1, 2, 3), replicas.replicas()); PartitionChangeRecord expectedRecord = new PartitionChangeRecord(). setTopicId(FOO_ID). setPartitionId(0). - setReplicas(Arrays.asList(1, 2)). - setIsr(Arrays.asList(2, 1)). + setReplicas(List.of(1, 2)). + setIsr(List.of(2, 1)). setLeader(1); if (version >= 1) { Map dirs = DirectoryId.createAssignmentMap(FOO.replicas, FOO.directories); - expectedRecord.setDirectories(Arrays.asList(dirs.get(1), dirs.get(2))); + expectedRecord.setDirectories(List.of(dirs.get(1), dirs.get(2))); } assertEquals(Optional.of(new ApiMessageAndVersion(expectedRecord, version)), createFooBuilder(version). @@ -534,18 +533,18 @@ public class PartitionChangeBuilderTest { @MethodSource("partitionChangeRecordVersions") public void testAddingReplicaReassignment(short version) { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(Arrays.asList(1, 2, 3, 4))); - assertEquals(Collections.emptyList(), replicas.removing()); - assertEquals(Collections.singletonList(4), replicas.adding()); - assertEquals(Arrays.asList(1, 2, 3, 4), replicas.replicas()); + partitionAssignment(Replicas.toList(FOO.replicas)), partitionAssignment(List.of(1, 2, 3, 4))); + assertEquals(List.of(), replicas.removing()); + assertEquals(List.of(4), replicas.adding()); + assertEquals(List.of(1, 2, 3, 4), replicas.replicas()); PartitionChangeRecord expectedRecord = new PartitionChangeRecord(). setTopicId(FOO_ID). setPartitionId(0). - setReplicas(Arrays.asList(1, 2, 3, 4)). - setAddingReplicas(Collections.singletonList(4)); + setReplicas(List.of(1, 2, 3, 4)). + setAddingReplicas(List.of(4)); if (version >= 1) { Map dirs = DirectoryId.createAssignmentMap(FOO.replicas, FOO.directories); - expectedRecord.setDirectories(Arrays.asList(dirs.get(1), dirs.get(2), dirs.get(3), DirectoryId.UNASSIGNED)); + expectedRecord.setDirectories(List.of(dirs.get(1), dirs.get(2), dirs.get(3), DirectoryId.UNASSIGNED)); } assertEquals(Optional.of(new ApiMessageAndVersion(expectedRecord, version)), createFooBuilder(version). @@ -561,7 +560,7 @@ public class PartitionChangeBuilderTest { new PartitionChangeRecord() .setTopicId(FOO_ID) .setPartitionId(0) - .setIsr(Collections.singletonList(2)) + .setIsr(List.of(2)) .setLeader(2) .setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()), version @@ -569,20 +568,20 @@ public class PartitionChangeBuilderTest { assertEquals( Optional.of(expectedRecord), createFooBuilder(version).setElection(Election.UNCLEAN) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Collections.singletonList(3))).build() + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(3))).build() ); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(OFFLINE_ID) .setPartitionId(0) - .setIsr(Collections.singletonList(1)) + .setIsr(List.of(1)) .setLeader(1) .setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()); if (version >= 2) { // The test partition has ELR, so unclean election will clear these fields. - record.setEligibleLeaderReplicas(Collections.emptyList()) - .setLastKnownElr(Collections.emptyList()); + record.setEligibleLeaderReplicas(List.of()) + .setLastKnownElr(List.of()); } expectedRecord = new ApiMessageAndVersion(record, version); @@ -594,7 +593,7 @@ public class PartitionChangeBuilderTest { assertEquals( Optional.of(expectedRecord), createOfflineBuilder(version).setElection(Election.UNCLEAN) - .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Collections.singletonList(2))).build() + .setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(2))).build() ); } @@ -629,7 +628,7 @@ public class PartitionChangeBuilderTest { 2 ); // Set the target ISR to empty to indicate that the last leader is offline - offlineBuilder.setTargetIsrWithBrokerStates(Collections.emptyList()); + offlineBuilder.setTargetIsrWithBrokerStates(List.of()); // The partition should stay as recovering PartitionChangeRecord changeRecord = (PartitionChangeRecord) offlineBuilder @@ -767,13 +766,13 @@ public class PartitionChangeBuilderTest { assertEquals(Optional.of(new ApiMessageAndVersion(new PartitionChangeRecord(). setTopicId(topicId). setPartitionId(0). - setReplicas(Arrays.asList(2, 3)). - setIsr(Arrays.asList(2, 3)). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()). + setReplicas(List.of(2, 3)). + setIsr(List.of(2, 3)). + setRemovingReplicas(List.of()). + setAddingReplicas(List.of()). setLeader(NO_LEADER), (short) 0)), - partitionChangeBuilder.setTargetIsr(Arrays.asList(0, 1, 2, 3)). + partitionChangeBuilder.setTargetIsr(List.of(0, 1, 2, 3)). build()); } @@ -803,16 +802,16 @@ public class PartitionChangeBuilderTest { .setUseLastKnownLeaderInBalancedRecovery(false); // Update ISR to {1, 2} - builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2))); + builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 2))); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Arrays.asList(1, 2)) + .setIsr(List.of(1, 2)) .setLeader(-2) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE); if (version >= 2) { - record.setEligibleLeaderReplicas(Arrays.asList(3, 4)); + record.setEligibleLeaderReplicas(List.of(3, 4)); } ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion(record, version); assertEquals(Optional.of(expectedRecord), builder.build()); @@ -854,17 +853,17 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(false); - builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3))); + builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 2, 3))); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Arrays.asList(1, 2, 3)) + .setIsr(List.of(1, 2, 3)) .setLeader(-2) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE); // Both versions will set the elr and lastKnownElr as empty list. - record.setEligibleLeaderReplicas(Collections.emptyList()) - .setLastKnownElr(Collections.emptyList()); + record.setEligibleLeaderReplicas(List.of()) + .setLastKnownElr(List.of()); ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion(record, version); assertEquals(Optional.of(expectedRecord), builder.build()); partition = partition.merge((PartitionChangeRecord) builder.build().get().message()); @@ -900,16 +899,16 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(false); - builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 4))); + builder.setTargetIsrWithBrokerStates(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 4))); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Arrays.asList(1, 4)) + .setIsr(List.of(1, 4)) .setLeader(-2) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE); if (version < 2) { - record.setEligibleLeaderReplicas(Collections.emptyList()); - record.setLastKnownElr(Collections.emptyList()); + record.setEligibleLeaderReplicas(List.of()); + record.setLastKnownElr(List.of()); } // No change is expected to ELR/LastKnownElr. ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion(record, version); @@ -952,7 +951,7 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(false); - builder.setUncleanShutdownReplicas(Collections.singletonList(3)); + builder.setUncleanShutdownReplicas(List.of(3)); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(topicId) @@ -960,10 +959,10 @@ public class PartitionChangeBuilderTest { .setLeader(-2) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE); if (version >= 2) { - record.setEligibleLeaderReplicas(Collections.singletonList(2)) - .setLastKnownElr(Collections.singletonList(3)); + record.setEligibleLeaderReplicas(List.of(2)) + .setLastKnownElr(List.of(3)); } else { - record.setEligibleLeaderReplicas(Collections.emptyList()); + record.setEligibleLeaderReplicas(List.of()); } ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion(record, version); assertEquals(Optional.of(expectedRecord), builder.build()); @@ -994,7 +993,7 @@ public class PartitionChangeBuilderTest { build(); Optional built = new PartitionChangeBuilder(registration, FOO_ID, 0, r -> true, MetadataVersion.IBP_3_7_IV2, 2). - setTargetReplicas(Arrays.asList(3, 1, 5, 4)). + setTargetReplicas(List.of(3, 1, 5, 4)). setDirectory(5, Uuid.fromString("RNJ5oFjjSSWMMFRwqdCfJg")). setDefaultDirProvider(DEFAULT_DIR_PROVIDER). build(); @@ -1003,8 +1002,8 @@ public class PartitionChangeBuilderTest { setTopicId(FOO_ID). setPartitionId(0). setLeader(1). - setReplicas(Arrays.asList(3, 1, 5, 4)). - setDirectories(Arrays.asList( + setReplicas(List.of(3, 1, 5, 4)). + setDirectories(List.of( Uuid.fromString("fM5NKyWTQHqEihjIkUl99Q"), Uuid.fromString("iU2znv45Q9yQkOpkTSy3jA"), Uuid.fromString("RNJ5oFjjSSWMMFRwqdCfJg"), @@ -1040,7 +1039,7 @@ public class PartitionChangeBuilderTest { new PartitionChangeRecord(). setTopicId(FOO_ID). setPartitionId(0). - setDirectories(Arrays.asList( + setDirectories(List.of( Uuid.fromString("S1zMYZczRjWmucidLqGA5g"), DirectoryId.LOST, Uuid.fromString("pN1VKs9zRzK4APflpegAVg") @@ -1075,14 +1074,14 @@ public class PartitionChangeBuilderTest { .setUseLastKnownLeaderInBalancedRecovery(lastKnownLeaderEnabled) .setDefaultDirProvider(DEFAULT_DIR_PROVIDER); - builder.setTargetIsr(Collections.emptyList()); + builder.setTargetIsr(List.of()); ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion( new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Collections.singletonList(3)) - .setEligibleLeaderReplicas(Collections.singletonList(1)) + .setIsr(List.of(3)) + .setEligibleLeaderReplicas(List.of(1)) .setLeader(3) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE), version @@ -1124,18 +1123,18 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(lastKnownLeaderEnabled); - builder.setTargetIsr(Collections.emptyList()); + builder.setTargetIsr(List.of()); PartitionChangeRecord record = new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Collections.emptyList()) + .setIsr(List.of()) .setLeader(-1) .setLeaderRecoveryState(LeaderRecoveryState.NO_CHANGE) - .setEligibleLeaderReplicas(Arrays.asList(1, 2, 3, 4)); + .setEligibleLeaderReplicas(List.of(1, 2, 3, 4)); if (lastKnownLeaderEnabled) { - record.setLastKnownElr(Collections.singletonList(1)); + record.setLastKnownElr(List.of(1)); } ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion(record, version); @@ -1148,7 +1147,7 @@ public class PartitionChangeBuilderTest { metadataVersionForPartitionChangeRecordVersion(version), 3) .setElection(Election.PREFERRED) .setEligibleLeaderReplicasEnabled(true) - .setUncleanShutdownReplicas(Collections.singletonList(2)) + .setUncleanShutdownReplicas(List.of(2)) .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(lastKnownLeaderEnabled); PartitionChangeRecord changeRecord = (PartitionChangeRecord) builder.build().get().message(); @@ -1182,16 +1181,16 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setEligibleLeaderReplicasEnabled(true); - builder.setTargetIsr(Collections.emptyList()); + builder.setTargetIsr(List.of()); ApiMessageAndVersion expectedRecord = new ApiMessageAndVersion( new PartitionChangeRecord() .setTopicId(topicId) .setPartitionId(0) - .setIsr(Collections.singletonList(1)) + .setIsr(List.of(1)) .setLeader(1) .setLeaderRecoveryState(LeaderRecoveryState.RECOVERING.value()) - .setLastKnownElr(Collections.emptyList()), + .setLastKnownElr(List.of()), version ); assertEquals(Optional.of(expectedRecord), builder.build()); @@ -1229,7 +1228,7 @@ public class PartitionChangeBuilderTest { .setDefaultDirProvider(DEFAULT_DIR_PROVIDER) .setUseLastKnownLeaderInBalancedRecovery(true); - builder.setTargetIsr(Collections.emptyList()); + builder.setTargetIsr(List.of()); // No change to the partition. assertEquals(Optional.empty(), builder.build()); diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java index 3f4491e3d87..1a9cd50bd1d 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentReplicasTest.java @@ -24,8 +24,7 @@ import org.apache.kafka.metadata.PartitionRegistration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import java.util.Optional; import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; @@ -39,96 +38,96 @@ public class PartitionReassignmentReplicasTest { @Test public void testNoneAddedOrRemoved() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 2, 1))); - assertEquals(Collections.emptyList(), replicas.removing()); - assertEquals(Collections.emptyList(), replicas.adding()); - assertEquals(Arrays.asList(3, 2, 1), replicas.replicas()); + partitionAssignment(List.of(3, 2, 1)), partitionAssignment(List.of(3, 2, 1))); + assertEquals(List.of(), replicas.removing()); + assertEquals(List.of(), replicas.adding()); + assertEquals(List.of(3, 2, 1), replicas.replicas()); } @Test public void testAdditions() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(3, 2, 1)), partitionAssignment(Arrays.asList(3, 6, 2, 1, 5))); - assertEquals(Collections.emptyList(), replicas.removing()); - assertEquals(Arrays.asList(5, 6), replicas.adding()); - assertEquals(Arrays.asList(3, 6, 2, 1, 5), replicas.replicas()); + partitionAssignment(List.of(3, 2, 1)), partitionAssignment(List.of(3, 6, 2, 1, 5))); + assertEquals(List.of(), replicas.removing()); + assertEquals(List.of(5, 6), replicas.adding()); + assertEquals(List.of(3, 6, 2, 1, 5), replicas.replicas()); } @Test public void testRemovals() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(3, 1))); - assertEquals(Arrays.asList(0, 2), replicas.removing()); - assertEquals(Collections.emptyList(), replicas.adding()); - assertEquals(Arrays.asList(3, 1, 0, 2), replicas.replicas()); + partitionAssignment(List.of(3, 2, 1, 0)), partitionAssignment(List.of(3, 1))); + assertEquals(List.of(0, 2), replicas.removing()); + assertEquals(List.of(), replicas.adding()); + assertEquals(List.of(3, 1, 0, 2), replicas.replicas()); } @Test public void testAdditionsAndRemovals() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(7, 3, 1, 9))); - assertEquals(Arrays.asList(0, 2), replicas.removing()); - assertEquals(Arrays.asList(7, 9), replicas.adding()); - assertEquals(Arrays.asList(7, 3, 1, 9, 0, 2), replicas.replicas()); + partitionAssignment(List.of(3, 2, 1, 0)), partitionAssignment(List.of(7, 3, 1, 9))); + assertEquals(List.of(0, 2), replicas.removing()); + assertEquals(List.of(7, 9), replicas.adding()); + assertEquals(List.of(7, 3, 1, 9, 0, 2), replicas.replicas()); } @Test public void testRearrangement() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(3, 2, 1, 0)), partitionAssignment(Arrays.asList(0, 1, 3, 2))); - assertEquals(Collections.emptyList(), replicas.removing()); - assertEquals(Collections.emptyList(), replicas.adding()); - assertEquals(Arrays.asList(0, 1, 3, 2), replicas.replicas()); + partitionAssignment(List.of(3, 2, 1, 0)), partitionAssignment(List.of(0, 1, 3, 2))); + assertEquals(List.of(), replicas.removing()); + assertEquals(List.of(), replicas.adding()); + assertEquals(List.of(0, 1, 3, 2), replicas.replicas()); } @Test public void testDoesNotCompleteReassignment() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5))); + partitionAssignment(List.of(0, 1, 2)), partitionAssignment(List.of(3, 4, 5))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = - replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4)); + replicas.maybeCompleteReassignment(List.of(0, 1, 2, 3, 4)); assertFalse(reassignmentOptional.isPresent()); } @Test public void testDoesNotCompleteReassignmentIfNoneOngoing() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - Collections.emptyList(), - Collections.emptyList(), - Arrays.asList(0, 1, 2) + List.of(), + List.of(), + List.of(0, 1, 2) ); assertFalse(replicas.isReassignmentInProgress()); Optional reassignmentOptional = - replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2)); + replicas.maybeCompleteReassignment(List.of(0, 1, 2)); assertFalse(reassignmentOptional.isPresent()); } @Test public void testDoesCompleteReassignmentAllNewReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(3, 4, 5))); + partitionAssignment(List.of(0, 1, 2)), partitionAssignment(List.of(3, 4, 5))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = - replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3, 4, 5)); + replicas.maybeCompleteReassignment(List.of(0, 1, 2, 3, 4, 5)); assertTrue(reassignmentOptional.isPresent()); PartitionReassignmentReplicas.CompletedReassignment completedReassignment = reassignmentOptional.get(); - assertEquals(Arrays.asList(3, 4, 5), completedReassignment.isr); - assertEquals(Arrays.asList(3, 4, 5), completedReassignment.replicas); + assertEquals(List.of(3, 4, 5), completedReassignment.isr); + assertEquals(List.of(3, 4, 5), completedReassignment.replicas); } @Test public void testDoesCompleteReassignmentSomeNewReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); + partitionAssignment(List.of(0, 1, 2)), partitionAssignment(List.of(0, 1, 3))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = - replicas.maybeCompleteReassignment(Arrays.asList(0, 1, 2, 3)); + replicas.maybeCompleteReassignment(List.of(0, 1, 2, 3)); assertTrue(reassignmentOptional.isPresent()); PartitionReassignmentReplicas.CompletedReassignment completedReassignment = reassignmentOptional.get(); - assertEquals(Arrays.asList(0, 1, 3), completedReassignment.isr); - assertEquals(Arrays.asList(0, 1, 3), completedReassignment.replicas); + assertEquals(List.of(0, 1, 3), completedReassignment.isr); + assertEquals(List.of(0, 1, 3), completedReassignment.replicas); } @Test @@ -200,17 +199,17 @@ public class PartitionReassignmentReplicasTest { @Test public void testDoesNotCompleteReassignmentIfIsrDoesNotHaveAllTargetReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); + partitionAssignment(List.of(0, 1, 2)), partitionAssignment(List.of(0, 1, 3))); assertTrue(replicas.isReassignmentInProgress()); Optional reassignmentOptional = - replicas.maybeCompleteReassignment(Collections.singletonList(3)); + replicas.maybeCompleteReassignment(List.of(3)); assertFalse(reassignmentOptional.isPresent()); } @Test public void testOriginalReplicas() { PartitionReassignmentReplicas replicas = new PartitionReassignmentReplicas( - partitionAssignment(Arrays.asList(0, 1, 2)), partitionAssignment(Arrays.asList(0, 1, 3))); - assertEquals(Arrays.asList(0, 1, 2), replicas.originalReplicas()); + partitionAssignment(List.of(0, 1, 2)), partitionAssignment(List.of(0, 1, 3))); + assertEquals(List.of(0, 1, 2), replicas.originalReplicas()); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java index 721280b881d..d80736d2fc3 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PartitionReassignmentRevertTest.java @@ -24,7 +24,7 @@ import org.apache.kafka.metadata.PartitionRegistration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -44,8 +44,8 @@ public class PartitionReassignmentRevertTest { }). setLeader(3).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); - assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); - assertEquals(Arrays.asList(3, 2), revert.isr()); + assertEquals(List.of(3, 2, 1), revert.replicas()); + assertEquals(List.of(3, 2), revert.isr()); assertFalse(revert.unclean()); } @@ -60,8 +60,8 @@ public class PartitionReassignmentRevertTest { }). setRemovingReplicas(new int[]{2, 1}).setLeader(3).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); - assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); - assertEquals(Arrays.asList(3, 2), revert.isr()); + assertEquals(List.of(3, 2, 1), revert.replicas()); + assertEquals(List.of(3, 2), revert.isr()); assertFalse(revert.unclean()); } @@ -78,8 +78,8 @@ public class PartitionReassignmentRevertTest { }). setAddingReplicas(new int[]{4, 5}).setLeader(3).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); - assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); - assertEquals(Arrays.asList(2), revert.isr()); + assertEquals(List.of(3, 2, 1), revert.replicas()); + assertEquals(List.of(2), revert.isr()); assertFalse(revert.unclean()); } @@ -96,8 +96,8 @@ public class PartitionReassignmentRevertTest { }). setRemovingReplicas(new int[]{2}).setAddingReplicas(new int[]{4, 5}).setLeader(3).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); - assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); - assertEquals(Arrays.asList(2), revert.isr()); + assertEquals(List.of(3, 2, 1), revert.replicas()); + assertEquals(List.of(2), revert.isr()); assertFalse(revert.unclean()); } @@ -114,8 +114,8 @@ public class PartitionReassignmentRevertTest { }). setRemovingReplicas(new int[]{2}).setAddingReplicas(new int[]{4, 5}).setLeader(3).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionReassignmentRevert revert = new PartitionReassignmentRevert(registration); - assertEquals(Arrays.asList(3, 2, 1), revert.replicas()); - assertEquals(Arrays.asList(3), revert.isr()); + assertEquals(List.of(3, 2, 1), revert.replicas()); + assertEquals(List.of(3), revert.isr()); assertTrue(revert.unclean()); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java index 1defdec760c..aebdb596c6c 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/PeriodicTaskControlManagerTest.java @@ -23,7 +23,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Collections; import java.util.EnumSet; import java.util.Iterator; import java.util.List; @@ -57,7 +56,7 @@ public class PeriodicTaskControlManagerTest { if (shouldFail.getAndSet(false)) { throw new NullPointerException("uh oh"); } - return ControllerResult.of(Collections.emptyList(), + return ControllerResult.of(List.of(), continuation.getAndSet(false)); }, periodNs, diff --git a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java index a27f249a2e2..95a1d3a3020 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ProducerIdControlManagerTest.java @@ -30,7 +30,7 @@ import org.apache.kafka.timeline.SnapshotRegistry; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import java.util.Collections; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -48,7 +48,7 @@ public class ProducerIdControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); ClusterControlManager clusterControl = new ClusterControlManager.Builder(). setTime(time). diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java index 6057df09698..0157436ffb6 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerIntegrationTestUtils.java @@ -35,7 +35,6 @@ import org.apache.kafka.server.common.MetadataVersion; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -124,11 +123,11 @@ public class QuorumControllerIntegrationTestUtils { .setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))) .setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + brokerId)) - .setLogDirs(Collections.singletonList( + .setLogDirs(List.of( Uuid.fromString("TESTBROKER" + Integer.toString(100000 + brokerId).substring(1) + "DIRAAAA") )) .setListeners(new ListenerCollection( - Collections.singletonList( + List.of( new Listener() .setName("PLAINTEXT") .setHost("localhost") diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java index ceae7a511e1..3c2ae307955 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTest.java @@ -126,7 +126,6 @@ import java.io.File; import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -147,7 +146,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static java.util.Collections.singletonList; import static java.util.function.Function.identity; import static org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; @@ -194,7 +192,7 @@ public class QuorumControllerTest { setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setBrokerId(0). - setLogDirs(Collections.singletonList(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). + setLogDirs(List.of(Uuid.fromString("iiaQjkRPQcuMULNII0MUeA"))). setClusterId(logEnv.clusterId())).get(); testConfigurationOperations(controlEnv.activeController()); @@ -203,20 +201,19 @@ public class QuorumControllerTest { } private void testConfigurationOperations(QuorumController controller) throws Throwable { - assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE), - controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), true).get()); - assertEquals(Collections.singletonMap(BROKER0, - new ResultOrError<>(Collections.emptyMap())), - controller.describeConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.emptyList())).get()); - assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE), - controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), false).get()); - assertEquals(Collections.singletonMap(BROKER0, new ResultOrError<>(Collections. - singletonMap("baz", "123"))), - controller.describeConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.emptyList())).get()); + assertEquals(Map.of(BROKER0, ApiError.NONE), + controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, Map.of("baz", entry(SET, "123"))), true).get()); + assertEquals(Map.of(BROKER0, + new ResultOrError<>(Map.of())), + controller.describeConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, List.of())).get()); + assertEquals(Map.of(BROKER0, ApiError.NONE), + controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, Map.of("baz", entry(SET, "123"))), false).get()); + assertEquals(Map.of(BROKER0, new ResultOrError<>(Map.of("baz", "123"))), + controller.describeConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, List.of())).get()); } /** @@ -236,7 +233,7 @@ public class QuorumControllerTest { setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setBrokerId(0). - setLogDirs(Collections.singletonList(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). + setLogDirs(List.of(Uuid.fromString("sTbzRAMnTpahIyIPNjiLhw"))). setClusterId(logEnv.clusterId())).get(); testDelayedConfigurationOperations(logEnv, controlEnv.activeController()); @@ -250,22 +247,22 @@ public class QuorumControllerTest { ) throws Throwable { logEnv.logManagers().forEach(m -> m.setMaxReadOffset(1L)); CompletableFuture> future1 = - controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.singletonMap("baz", entry(SET, "123"))), false); + controller.incrementalAlterConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, Map.of("baz", entry(SET, "123"))), false); assertFalse(future1.isDone()); - assertEquals(Collections.singletonMap(BROKER0, - new ResultOrError<>(Collections.emptyMap())), - controller.describeConfigs(ANONYMOUS_CONTEXT, Collections.singletonMap( - BROKER0, Collections.emptyList())).get()); + assertEquals(Map.of(BROKER0, + new ResultOrError<>(Map.of())), + controller.describeConfigs(ANONYMOUS_CONTEXT, Map.of( + BROKER0, List.of())).get()); logEnv.logManagers().forEach(m -> m.setMaxReadOffset(8L)); - assertEquals(Collections.singletonMap(BROKER0, ApiError.NONE), future1.get()); + assertEquals(Map.of(BROKER0, ApiError.NONE), future1.get()); } @Test public void testFenceMultipleBrokers() throws Throwable { - List allBrokers = Arrays.asList(1, 2, 3, 4, 5); - List brokersToKeepUnfenced = Arrays.asList(1); - List brokersToFence = Arrays.asList(2, 3, 4, 5); + List allBrokers = List.of(1, 2, 3, 4, 5); + List brokersToKeepUnfenced = List.of(1); + List brokersToFence = List.of(2, 3, 4, 5); short replicationFactor = (short) allBrokers.size(); short numberOfPartitions = (short) allBrokers.size(); long sessionTimeoutMillis = 1000; @@ -303,12 +300,12 @@ public class QuorumControllerTest { // Unfence all brokers and create a topic foo sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions). setReplicationFactor(replicationFactor)).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics( ANONYMOUS_CONTEXT, createTopicsRequestData, - Collections.singleton("foo")).get(); + Set.of("foo")).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); @@ -362,7 +359,7 @@ public class QuorumControllerTest { QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv). setSessionTimeoutMillis(OptionalLong.of(sessionTimeoutMillis)). setBootstrapMetadata(BootstrapMetadata.fromRecords( - Arrays.asList( + List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.IBP_4_0_IV1.featureLevel()), (short) 0), @@ -382,7 +379,7 @@ public class QuorumControllerTest { @Flaky("KAFKA-18845") @Test public void testUncleanShutdownBrokerElrEnabled() throws Throwable { - List allBrokers = Arrays.asList(1, 2, 3); + List allBrokers = List.of(1, 2, 3); short replicationFactor = (short) allBrokers.size(); long sessionTimeoutMillis = 500; @@ -409,7 +406,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setFeatures(features). setIncarnationId(Uuid.randomUuid()). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); } @@ -422,12 +419,12 @@ public class QuorumControllerTest { // Unfence all brokers and create a topic foo sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(1). setReplicationFactor(replicationFactor)).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics( ANONYMOUS_CONTEXT, createTopicsRequestData, - Collections.singleton("foo")).get(); + Set.of("foo")).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); ConfigRecord configRecord = new ConfigRecord() @@ -435,7 +432,7 @@ public class QuorumControllerTest { .setResourceName("foo") .setName(org.apache.kafka.common.config.TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG) .setValue("2"); - RecordTestUtils.replayAll(active.configurationControl(), singletonList(new ApiMessageAndVersion(configRecord, (short) 0))); + RecordTestUtils.replayAll(active.configurationControl(), List.of(new ApiMessageAndVersion(configRecord, (short) 0))); // Fence all the brokers TestUtils.waitForCondition(() -> { @@ -477,7 +474,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setFeatures(features). setIncarnationId(Uuid.randomUuid()). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)); brokerEpochs.put(brokerToUncleanShutdown, reply.get().epoch()); partition = active.replicationControl().getPartition(topicIdFoo, 0); @@ -493,11 +490,11 @@ public class QuorumControllerTest { setFeatures(features). setIncarnationId(Uuid.randomUuid()). setPreviousBrokerEpoch(brokerEpochs.get(brokerToBeTheLeader)). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)); // Unfence the last one in the ELR, it should be elected. - sendBrokerHeartbeatToUnfenceBrokers(active, singletonList(brokerToBeTheLeader), brokerEpochs); + sendBrokerHeartbeatToUnfenceBrokers(active, List.of(brokerToBeTheLeader), brokerEpochs); TestUtils.waitForCondition(() -> active.clusterControl().isUnfenced(brokerToBeTheLeader), sessionTimeoutMillis * 3, "Broker should be unfenced." ); @@ -512,7 +509,7 @@ public class QuorumControllerTest { @Test public void testUncleanShutdownElrDisabled() throws Exception { - List allBrokers = Arrays.asList(1, 2, 3); + List allBrokers = List.of(1, 2, 3); short replicationFactor = (short) allBrokers.size(); long sessionTimeoutMillis = 500; @@ -541,7 +538,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setFeatures(features). setIncarnationId(Uuid.randomUuid()). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); } @@ -554,12 +551,12 @@ public class QuorumControllerTest { // Unfence all brokers and create a topic foo sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(1). setReplicationFactor(replicationFactor)).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics( ANONYMOUS_CONTEXT, createTopicsRequestData, - Collections.singleton("foo")).get(); + Set.of("foo")).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); @@ -584,7 +581,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setFeatures(features). setIncarnationId(Uuid.randomUuid()). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)).get(); partition = active.replicationControl().getPartition(topicIdFoo, 0); // leader should always change, leader epoch should always be incremented @@ -606,9 +603,9 @@ public class QuorumControllerTest { @Test public void testMinIsrUpdateWithElr() throws Throwable { - List allBrokers = Arrays.asList(1, 2, 3); - List brokersToKeepUnfenced = Arrays.asList(1); - List brokersToFence = Arrays.asList(2, 3); + List allBrokers = List.of(1, 2, 3); + List brokersToKeepUnfenced = List.of(1); + List brokersToFence = List.of(2, 3); short replicationFactor = (short) allBrokers.size(); long sessionTimeoutMillis = 300; @@ -633,7 +630,7 @@ public class QuorumControllerTest { setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). setIncarnationId(Uuid.randomUuid()). - setLogDirs(Collections.singletonList(Uuid.randomUuid())). + setLogDirs(List.of(Uuid.randomUuid())). setListeners(listeners)); brokerEpochs.put(brokerId, reply.get().epoch()); } @@ -647,7 +644,7 @@ public class QuorumControllerTest { // Unfence all brokers and create a topic foo (min ISR 2) sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Arrays.asList( + new CreatableTopicCollection(List.of( new CreatableTopic().setName("foo").setNumPartitions(1). setReplicationFactor(replicationFactor), new CreatableTopic().setName("bar").setNumPartitions(1). @@ -655,7 +652,7 @@ public class QuorumControllerTest { ).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics( ANONYMOUS_CONTEXT, createTopicsRequestData, - new HashSet<>(Arrays.asList("foo", "bar"))).get(); + new HashSet<>(List.of("foo", "bar"))).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("bar").errorCode())); Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); @@ -665,7 +662,7 @@ public class QuorumControllerTest { .setResourceName("") .setName(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG) .setValue("2"); - RecordTestUtils.replayAll(active.configurationControl(), singletonList(new ApiMessageAndVersion(configRecord, (short) 0))); + RecordTestUtils.replayAll(active.configurationControl(), List.of(new ApiMessageAndVersion(configRecord, (short) 0))); // Fence brokers TestUtils.waitForCondition(() -> { @@ -706,8 +703,8 @@ public class QuorumControllerTest { entry(new ConfigResource(TOPIC, "foo"), toMap(entry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, entry(SET, "1"))))), true); assertEquals(2, result.records().size(), result.records().toString()); - RecordTestUtils.replayAll(active.configurationControl(), singletonList(result.records().get(0))); - RecordTestUtils.replayAll(active.replicationControl(), singletonList(result.records().get(1))); + RecordTestUtils.replayAll(active.configurationControl(), List.of(result.records().get(0))); + RecordTestUtils.replayAll(active.replicationControl(), List.of(result.records().get(1))); partition = active.replicationControl().getPartition(topicIdFoo, 0); assertEquals(0, partition.elr.length, partition.toString()); @@ -722,8 +719,8 @@ public class QuorumControllerTest { entry(new ConfigResource(BROKER, ""), toMap(entry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, entry(SET, "1"))))), true); assertEquals(2, result.records().size(), result.records().toString()); - RecordTestUtils.replayAll(active.configurationControl(), singletonList(result.records().get(0))); - RecordTestUtils.replayAll(active.replicationControl(), singletonList(result.records().get(1))); + RecordTestUtils.replayAll(active.configurationControl(), List.of(result.records().get(0))); + RecordTestUtils.replayAll(active.replicationControl(), List.of(result.records().get(1))); partition = active.replicationControl().getPartition(topicIdBar, 0); assertEquals(0, partition.elr.length, partition.toString()); @@ -733,9 +730,9 @@ public class QuorumControllerTest { @Test public void testBalancePartitionLeaders() throws Throwable { - List allBrokers = Arrays.asList(1, 2, 3); - List brokersToKeepUnfenced = Arrays.asList(1, 2); - List brokersToFence = Collections.singletonList(3); + List allBrokers = List.of(1, 2, 3); + List brokersToKeepUnfenced = List.of(1, 2); + List brokersToFence = List.of(3); short replicationFactor = (short) allBrokers.size(); short numberOfPartitions = (short) allBrokers.size(); long sessionTimeoutMillis = 2000; @@ -775,11 +772,11 @@ public class QuorumControllerTest { // Unfence all brokers and create a topic foo sendBrokerHeartbeatToUnfenceBrokers(active, allBrokers, brokerEpochs); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(numberOfPartitions). setReplicationFactor(replicationFactor)).iterator())); CreateTopicsResponseData createTopicsResponseData = active.createTopics( - ANONYMOUS_CONTEXT, createTopicsRequestData, Collections.singleton("foo")).get(); + ANONYMOUS_CONTEXT, createTopicsRequestData, Set.of("foo")).get(); assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode())); Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId(); @@ -838,7 +835,7 @@ public class QuorumControllerTest { .setPartitionIndex(imbalancedPartitionId) .setLeaderEpoch(partitionRegistration.leaderEpoch) .setPartitionEpoch(partitionRegistration.partitionEpoch) - .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(Arrays.asList(1, 2, 3))); + .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(List.of(1, 2, 3))); AlterPartitionRequestData.TopicData topicData = new AlterPartitionRequestData.TopicData() .setTopicId(impalancedTp.topicId()); @@ -954,7 +951,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). setFeatures(brokerFeatures). - setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). + setLogDirs(List.of(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners); if (brokerMaxSupportedKraftVersion < finalizedKraftVersion) { @@ -995,28 +992,28 @@ public class QuorumControllerTest { setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwBA")). setFeatures(brokerFeaturesPlusFeatureVersions(MetadataVersion.MINIMUM_VERSION, MetadataVersion.latestTesting(), Map.of(EligibleLeaderReplicasVersion.FEATURE_NAME, EligibleLeaderReplicasVersion.ELRV_1.featureLevel()))). - setLogDirs(Collections.singletonList(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). + setLogDirs(List.of(Uuid.fromString("vBpaRsZVSaGsQT53wtYGtg"))). setListeners(listeners)); assertEquals(6L, reply.get().epoch()); CreateTopicsRequestData createTopicsRequestData = new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(1). setReplicationFactor((short) 1)).iterator())); assertEquals(Errors.INVALID_REPLICATION_FACTOR.code(), active.createTopics( ANONYMOUS_CONTEXT, - createTopicsRequestData, Collections.singleton("foo")).get(). + createTopicsRequestData, Set.of("foo")).get(). topics().find("foo").errorCode()); assertEquals("Unable to replicate the partition 1 time(s): All brokers " + "are currently fenced.", active.createTopics(ANONYMOUS_CONTEXT, - createTopicsRequestData, Collections.singleton("foo")). + createTopicsRequestData, Set.of("foo")). get().topics().find("foo").errorMessage()); assertEquals(new BrokerHeartbeatReply(true, false, false, false), active.processBrokerHeartbeat(ANONYMOUS_CONTEXT, new BrokerHeartbeatRequestData(). setWantFence(false).setBrokerEpoch(6L).setBrokerId(0). setCurrentMetadataOffset(100000L)).get()); assertEquals(Errors.NONE.code(), active.createTopics(ANONYMOUS_CONTEXT, - createTopicsRequestData, Collections.singleton("foo")). + createTopicsRequestData, Set.of("foo")). get().topics().find("foo").errorCode()); CompletableFuture topicPartitionFuture = active.appendReadEvent( "debugGetPartition", OptionalLong.empty(), () -> { @@ -1072,7 +1069,7 @@ public class QuorumControllerTest { setIncarnationId(new Uuid(3465346L, i)). setZkMigrationReady(false). setListeners(new ControllerRegistrationRequestData.ListenerCollection( - singletonList( + List.of( new ControllerRegistrationRequestData.Listener(). setName("CONTROLLER"). setHost("localhost"). @@ -1081,7 +1078,7 @@ public class QuorumControllerTest { ).iterator() )). setFeatures(new ControllerRegistrationRequestData.FeatureCollection( - singletonList( + List.of( new ControllerRegistrationRequestData.Feature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). @@ -1097,7 +1094,7 @@ public class QuorumControllerTest { setClusterId(active.clusterId()). setFeatures(brokerFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB" + i)). - setListeners(new ListenerCollection(singletonList(new Listener(). + setListeners(new ListenerCollection(List.of(new Listener(). setName("PLAINTEXT").setHost("localhost"). setPort(9092 + i)).iterator()))).get(); brokerEpochs.put(i, reply.epoch()); @@ -1110,18 +1107,18 @@ public class QuorumControllerTest { } CreateTopicsResponseData fooData = active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().setTopics( - new CreatableTopicCollection(Collections.singleton( + new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo").setNumPartitions(-1). setReplicationFactor((short) -1). setAssignments(new CreatableReplicaAssignmentCollection( - Arrays.asList(new CreatableReplicaAssignment(). + List.of(new CreatableReplicaAssignment(). setPartitionIndex(0). - setBrokerIds(Arrays.asList(0, 1, 2)), + setBrokerIds(List.of(0, 1, 2)), new CreatableReplicaAssignment(). setPartitionIndex(1). - setBrokerIds(Arrays.asList(1, 2, 0))). + setBrokerIds(List.of(1, 2, 0))). iterator()))).iterator())), - Collections.singleton("foo")).get(); + Set.of("foo")).get(); fooId = fooData.topics().find("foo").topicId(); active.allocateProducerIds(ANONYMOUS_CONTEXT, new AllocateProducerIdsRequestData().setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0))).get(); @@ -1133,7 +1130,7 @@ public class QuorumControllerTest { } private List generateTestRecords(Uuid fooId, Map brokerEpochs) { - return Arrays.asList( + return List.of( new ApiMessageAndVersion(new BeginTransactionRecord(). setName("Bootstrap records"), (short) 0), new ApiMessageAndVersion(new FeatureLevelRecord(). @@ -1144,14 +1141,14 @@ public class QuorumControllerTest { setControllerId(0). setIncarnationId(Uuid.fromString("AAAAAAA04IIAAAAAAAAAAA")). setEndPoints(new RegisterControllerRecord.ControllerEndpointCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerEndpoint(). setName("CONTROLLER"). setHost("localhost"). setPort(8000). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())). setFeatures(new RegisterControllerRecord.ControllerFeatureCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). @@ -1161,14 +1158,14 @@ public class QuorumControllerTest { setControllerId(1). setIncarnationId(Uuid.fromString("AAAAAAA04IIAAAAAAAAAAQ")). setEndPoints(new RegisterControllerRecord.ControllerEndpointCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerEndpoint(). setName("CONTROLLER"). setHost("localhost"). setPort(8001). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())). setFeatures(new RegisterControllerRecord.ControllerFeatureCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). @@ -1178,14 +1175,14 @@ public class QuorumControllerTest { setControllerId(2). setIncarnationId(Uuid.fromString("AAAAAAA04IIAAAAAAAAAAg")). setEndPoints(new RegisterControllerRecord.ControllerEndpointCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerEndpoint(). setName("CONTROLLER"). setHost("localhost"). setPort(8002). setSecurityProtocol(SecurityProtocol.PLAINTEXT.id)).iterator())). setFeatures(new RegisterControllerRecord.ControllerFeatureCollection( - singletonList( + List.of( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). @@ -1195,7 +1192,7 @@ public class QuorumControllerTest { setBrokerId(0).setBrokerEpoch(brokerEpochs.get(0)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB0")). setEndPoints(new BrokerEndpointCollection( - singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + List.of(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9092).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). @@ -1203,7 +1200,7 @@ public class QuorumControllerTest { new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(1).setBrokerEpoch(brokerEpochs.get(1)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB1")). - setEndPoints(new BrokerEndpointCollection(singletonList( + setEndPoints(new BrokerEndpointCollection(List.of( new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9093).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). @@ -1213,7 +1210,7 @@ public class QuorumControllerTest { setBrokerId(2).setBrokerEpoch(brokerEpochs.get(2)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB2")). setEndPoints(new BrokerEndpointCollection( - singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + List.of(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9094).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). setRack(null). @@ -1221,7 +1218,7 @@ public class QuorumControllerTest { new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(3).setBrokerEpoch(brokerEpochs.get(3)). setIncarnationId(Uuid.fromString("kxAT73dKQsitIedpiPtwB3")). - setEndPoints(new BrokerEndpointCollection(singletonList( + setEndPoints(new BrokerEndpointCollection(List.of( new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9095).setSecurityProtocol((short) 0)).iterator())). setFeatures(registrationFeatures(MetadataVersion.MINIMUM_VERSION, MetadataVersion.IBP_3_7_IV0)). @@ -1242,14 +1239,14 @@ public class QuorumControllerTest { new ApiMessageAndVersion(new TopicRecord(). setName("foo").setTopicId(fooId), (short) 0), new ApiMessageAndVersion(new PartitionRecord().setPartitionId(0). - setTopicId(fooId).setReplicas(Arrays.asList(0, 1, 2)). - setIsr(Arrays.asList(0, 1, 2)).setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()).setLeader(0).setLeaderEpoch(0). + setTopicId(fooId).setReplicas(List.of(0, 1, 2)). + setIsr(List.of(0, 1, 2)).setRemovingReplicas(List.of()). + setAddingReplicas(List.of()).setLeader(0).setLeaderEpoch(0). setPartitionEpoch(0), (short) 0), new ApiMessageAndVersion(new PartitionRecord().setPartitionId(1). - setTopicId(fooId).setReplicas(Arrays.asList(1, 2, 0)). - setIsr(Arrays.asList(1, 2, 0)).setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()).setLeader(1).setLeaderEpoch(0). + setTopicId(fooId).setReplicas(List.of(1, 2, 0)). + setIsr(List.of(1, 2, 0)).setRemovingReplicas(List.of()). + setAddingReplicas(List.of()).setLeader(1).setLeaderEpoch(0). setPartitionEpoch(0), (short) 0), new ApiMessageAndVersion(new ProducerIdsRecord(). setBrokerId(0). @@ -1275,17 +1272,17 @@ public class QuorumControllerTest { new RequestHeaderData(), KafkaPrincipal.ANONYMOUS, OptionalLong.of(now)); CompletableFuture createFuture = controller.createTopics(context0, new CreateTopicsRequestData().setTimeoutMs(0). - setTopics(new CreatableTopicCollection(Collections.singleton( + setTopics(new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo")).iterator())), - Collections.emptySet()); + Set.of()); CompletableFuture> deleteFuture = - controller.deleteTopics(context0, Collections.singletonList(Uuid.ZERO_UUID)); + controller.deleteTopics(context0, List.of(Uuid.ZERO_UUID)); CompletableFuture>> findTopicIdsFuture = - controller.findTopicIds(context0, Collections.singletonList("foo")); + controller.findTopicIds(context0, List.of("foo")); CompletableFuture>> findTopicNamesFuture = - controller.findTopicNames(context0, Collections.singletonList(Uuid.ZERO_UUID)); + controller.findTopicNames(context0, List.of(Uuid.ZERO_UUID)); CompletableFuture> createPartitionsFuture = - controller.createPartitions(context0, Collections.singletonList( + controller.createPartitions(context0, List.of( new CreatePartitionsTopic()), false); CompletableFuture electLeadersFuture = controller.electLeaders(context0, new ElectLeadersRequestData().setTimeoutMs(0). @@ -1293,7 +1290,7 @@ public class QuorumControllerTest { CompletableFuture alterReassignmentsFuture = controller.alterPartitionReassignments(context0, new AlterPartitionReassignmentsRequestData().setTimeoutMs(0). - setTopics(Collections.singletonList(new ReassignableTopic()))); + setTopics(List.of(new ReassignableTopic()))); CompletableFuture listReassignmentsFuture = controller.listPartitionReassignments(context0, new ListPartitionReassignmentsRequestData().setTopics(null).setTimeoutMs(0)); @@ -1334,15 +1331,15 @@ public class QuorumControllerTest { CountDownLatch countDownLatch = pause(controller); CompletableFuture createFuture = controller.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData(). - setTimeoutMs(120000), Collections.emptySet()); + setTimeoutMs(120000), Set.of()); CompletableFuture> deleteFuture = - controller.deleteTopics(ANONYMOUS_CONTEXT, Collections.emptyList()); + controller.deleteTopics(ANONYMOUS_CONTEXT, List.of()); CompletableFuture>> findTopicIdsFuture = - controller.findTopicIds(ANONYMOUS_CONTEXT, Collections.emptyList()); + controller.findTopicIds(ANONYMOUS_CONTEXT, List.of()); CompletableFuture>> findTopicNamesFuture = - controller.findTopicNames(ANONYMOUS_CONTEXT, Collections.emptyList()); + controller.findTopicNames(ANONYMOUS_CONTEXT, List.of()); CompletableFuture> createPartitionsFuture = - controller.createPartitions(ANONYMOUS_CONTEXT, Collections.emptyList(), false); + controller.createPartitions(ANONYMOUS_CONTEXT, List.of(), false); CompletableFuture electLeadersFuture = controller.electLeaders(ANONYMOUS_CONTEXT, new ElectLeadersRequestData()); CompletableFuture alterReassignmentsFuture = @@ -1372,11 +1369,11 @@ public class QuorumControllerTest { QuorumController active = controlEnv.activeController(); registerBrokersAndUnfence(active, 5); active.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData(). - setTopics(new CreatableTopicCollection(Collections.singleton( + setTopics(new CreatableTopicCollection(Set.of( new CreatableTopic().setName("foo"). setReplicationFactor((short) 3). setNumPartitions(1)).iterator())), - Collections.singleton("foo")).get(); + Set.of("foo")).get(); ConfigResourceExistenceChecker checker = active.new ConfigResourceExistenceChecker(); // A ConfigResource with type=BROKER and name=(empty string) represents @@ -1416,7 +1413,7 @@ public class QuorumControllerTest { ) { QuorumController active = controlEnv.activeController(); CompletableFuture future = active.appendWriteEvent("errorEvent", - OptionalLong.empty(), () -> ControllerResult.of(Collections.singletonList(new ApiMessageAndVersion( + OptionalLong.empty(), () -> ControllerResult.of(List.of(new ApiMessageAndVersion( new ConfigRecord(). setName(null). setResourceName(null). @@ -1431,7 +1428,7 @@ public class QuorumControllerTest { @Test public void testFatalMetadataErrorDuringSnapshotLoading() throws Exception { - InitialSnapshot invalidSnapshot = new InitialSnapshot(singletonList( + InitialSnapshot invalidSnapshot = new InitialSnapshot(List.of( new ApiMessageAndVersion(new PartitionRecord(), (short) 0)) ); @@ -1455,7 +1452,7 @@ public class QuorumControllerTest { @Test public void testFatalMetadataErrorDuringLogLoading() throws Exception { try (LocalLogManagerTestEnv logEnv = new LocalLogManagerTestEnv.Builder(3).build()) { - logEnv.appendInitialRecords(singletonList( + logEnv.appendInitialRecords(List.of( new ApiMessageAndVersion(new PartitionRecord(), (short) 0))); try (QuorumControllerTestEnv controlEnv = new QuorumControllerTestEnv.Builder(logEnv).build()) { @@ -1496,7 +1493,7 @@ public class QuorumControllerTest { } private static final BootstrapMetadata COMPLEX_BOOTSTRAP = BootstrapMetadata.fromRecords( - Arrays.asList( + List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel()), @@ -1532,9 +1529,9 @@ public class QuorumControllerTest { TestUtils.waitForCondition(() -> { ConfigResource defaultBrokerResource = new ConfigResource(BROKER, ""); - Map> configs = Collections.singletonMap( + Map> configs = Map.of( defaultBrokerResource, - Collections.emptyList() + List.of() ); Map>> results = @@ -1542,7 +1539,7 @@ public class QuorumControllerTest { ResultOrError> resultOrError = results.get(defaultBrokerResource); return resultOrError.isResult() && - Collections.singletonMap("foo", "bar").equals(resultOrError.result()); + Map.of("foo", "bar").equals(resultOrError.result()); }, "Failed to see expected config change from bootstrap metadata"); testToImages(logEnv.allRecords()); @@ -1573,7 +1570,7 @@ public class QuorumControllerTest { public void testAppendRecords() { TestAppender appender = new TestAppender(); assertEquals(5, QuorumController.appendRecords(log, - ControllerResult.of(Arrays.asList(rec(0), rec(1), rec(2), rec(3), rec(4)), null), + ControllerResult.of(List.of(rec(0), rec(1), rec(2), rec(3), rec(4)), null), 2, appender)); } @@ -1584,7 +1581,7 @@ public class QuorumControllerTest { assertEquals("Attempted to atomically commit 5 records, but maxRecordsPerBatch is 2", assertThrows(IllegalStateException.class, () -> QuorumController.appendRecords(log, - ControllerResult.atomicOf(Arrays.asList(rec(0), rec(1), rec(2), rec(3), rec(4)), null), + ControllerResult.atomicOf(List.of(rec(0), rec(1), rec(2), rec(3), rec(4)), null), 2, appender)).getMessage()); } @@ -1654,7 +1651,7 @@ public class QuorumControllerTest { */ @SuppressWarnings("unchecked") private static void testToImages(List fromRecords) { - List> testMatrix = Arrays.asList( + List> testMatrix = List.of( new ImageDeltaPair<>(() -> AclsImage.EMPTY, AclsDelta::new), new ImageDeltaPair<>(() -> ClientQuotasImage.EMPTY, ClientQuotasDelta::new), new ImageDeltaPair<>(() -> ClusterImage.EMPTY, ClusterDelta::new), @@ -1679,7 +1676,7 @@ public class QuorumControllerTest { OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build(); offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, - Collections.singletonList(new ApiMessageAndVersion(new BeginTransactionRecord(), (short) 0)))); + List.of(new ApiMessageAndVersion(new BeginTransactionRecord(), (short) 0)))); ControllerResult result = ActivationRecordsGenerator.generate( logMsg -> { }, @@ -1701,7 +1698,7 @@ public class QuorumControllerTest { OffsetControlManager offsetControlManager = new OffsetControlManager.Builder().build(); offsetControlManager.replay(new BeginTransactionRecord(), 10); offsetControlManager.handleCommitBatch(Batch.data(20, 1, 1L, 0, - Collections.singletonList(new ApiMessageAndVersion(new BeginTransactionRecord(), (short) 0)))); + List.of(new ApiMessageAndVersion(new BeginTransactionRecord(), (short) 0)))); assertThrows(RuntimeException.class, () -> ActivationRecordsGenerator.generate( diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java index 925b41db79b..b04a3e8c658 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumControllerTestEnv.java @@ -35,7 +35,6 @@ import java.util.OptionalLong; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; -import java.util.stream.Collectors; import java.util.stream.IntStream; import static java.util.concurrent.TimeUnit.NANOSECONDS; @@ -101,7 +100,7 @@ public class QuorumControllerTestEnv implements AutoCloseable { int numControllers = logEnv.logManagers().size(); this.controllers = new ArrayList<>(numControllers); try { - List nodeIds = IntStream.range(0, numControllers).boxed().collect(Collectors.toList()); + List nodeIds = IntStream.range(0, numControllers).boxed().toList(); for (int nodeId = 0; nodeId < numControllers; nodeId++) { QuorumController.Builder builder = new QuorumController.Builder(nodeId, logEnv.clusterId()); builder.setRaftClient(logEnv.logManagers().get(nodeId)); diff --git a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java index 24e83a34cc5..19fdd87b918 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/QuorumFeaturesTest.java @@ -25,8 +25,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; @@ -43,7 +43,7 @@ public class QuorumFeaturesTest { ); private static final QuorumFeatures QUORUM_FEATURES = new QuorumFeatures(0, LOCAL, - Arrays.asList(0, 1, 2)); + List.of(0, 1, 2)); @Test public void testDefaultFeatureMap() { diff --git a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java index 7da1bd331b7..239c48e2069 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/ReplicationControlManagerTest.java @@ -112,7 +112,6 @@ import org.slf4j.LoggerFactory; import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -127,9 +126,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.IntStream; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; -import static java.util.Collections.singletonMap; import static org.apache.kafka.common.config.TopicConfig.SEGMENT_BYTES_CONFIG; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.protocol.Errors.ELECTION_NOT_NEEDED; @@ -240,7 +236,7 @@ public class ReplicationControlManagerTest { setSnapshotRegistry(snapshotRegistry). setQuorumFeatures(new QuorumFeatures(0, QuorumFeatures.defaultSupportedFeatureMap(true), - Collections.singletonList(0))). + List.of(0))). build(); this.featureControl.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). @@ -295,7 +291,7 @@ public class ReplicationControlManagerTest { request.topics().add(topic); ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton(name)); + replicationControl.createTopics(requestContext, request, Set.of(name)); CreatableTopicResult topicResult = result.response().topics().find(name); assertNotNull(topicResult); assertEquals(expectedErrorCode, topicResult.errorCode()); @@ -306,12 +302,12 @@ public class ReplicationControlManagerTest { } CreatableTopicResult createTestTopic(String name, int[][] replicas) { - return createTestTopic(name, replicas, Collections.emptyMap(), (short) 0); + return createTestTopic(name, replicas, Map.of(), (short) 0); } CreatableTopicResult createTestTopic(String name, int[][] replicas, short expectedErrorCode) { - return createTestTopic(name, replicas, Collections.emptyMap(), expectedErrorCode); + return createTestTopic(name, replicas, Map.of(), expectedErrorCode); } CreatableTopicResult createTestTopic(String name, int[][] replicas, @@ -334,7 +330,7 @@ public class ReplicationControlManagerTest { request.topics().add(topic); ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton(name)); + replicationControl.createTopics(requestContext, request, Set.of(name)); CreatableTopicResult topicResult = result.response().topics().find(name); assertNotNull(topicResult); assertEquals(expectedErrorCode, topicResult.errorCode()); @@ -347,8 +343,8 @@ public class ReplicationControlManagerTest { } void deleteTopic(ControllerRequestContext context, Uuid topicId) { - ControllerResult> result = replicationControl.deleteTopics(context, Collections.singleton(topicId)); - assertEquals(Collections.singleton(topicId), result.response().keySet()); + ControllerResult> result = replicationControl.deleteTopics(context, Set.of(topicId)); + assertEquals(Set.of(topicId), result.response().keySet()); assertEquals(NONE, result.response().get(topicId).error()); assertEquals(1, result.records().size()); @@ -372,7 +368,7 @@ public class ReplicationControlManagerTest { } ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_PARTITIONS); ControllerResult> result = - replicationControl.createPartitions(requestContext, Collections.singletonList(topic)); + replicationControl.createPartitions(requestContext, List.of(topic)); assertEquals(1, result.response().size()); CreatePartitionsTopicResult topicResult = result.response().get(0); assertEquals(name, topicResult.name()); @@ -384,7 +380,7 @@ public class ReplicationControlManagerTest { Object[] brokersAndDirs = new Object[brokerIds.length * 2]; for (int i = 0; i < brokerIds.length; i++) { brokersAndDirs[i * 2] = brokerIds[i]; - brokersAndDirs[i * 2 + 1] = Collections.singletonList( + brokersAndDirs[i * 2 + 1] = List.of( Uuid.fromString("TESTBROKER" + Integer.toString(100000 + brokerIds[i]).substring(1) + "DIRAAAA") ); } @@ -407,7 +403,7 @@ public class ReplicationControlManagerTest { setPort((short) 9092 + brokerId). setName("PLAINTEXT"). setHost("localhost")); - replay(Collections.singletonList(new ApiMessageAndVersion(brokerRecord, (short) 3))); + replay(List.of(new ApiMessageAndVersion(brokerRecord, (short) 3))); } } @@ -444,7 +440,7 @@ public class ReplicationControlManagerTest { TopicData topicData = new TopicData() .setTopicId(topicIdPartition.topicId()) - .setPartitions(singletonList(partitionData)); + .setPartitions(List.of(partitionData)); ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.ALTER_PARTITION); @@ -453,7 +449,7 @@ public class ReplicationControlManagerTest { new AlterPartitionRequestData() .setBrokerId(leaderId) .setBrokerEpoch(registration.epoch()) - .setTopics(singletonList(topicData))); + .setTopics(List.of(topicData))); replay(alterPartition.records()); } @@ -477,7 +473,7 @@ public class ReplicationControlManagerTest { .setBrokerId(brokerId) .setBrokerEpoch(defaultBrokerEpoch(brokerId)) .setInControlledShutdown(BrokerRegistrationInControlledShutdownChange.IN_CONTROLLED_SHUTDOWN.value()); - replay(singletonList(new ApiMessageAndVersion(record, (short) 1))); + replay(List.of(new ApiMessageAndVersion(record, (short) 1))); } } @@ -491,7 +487,7 @@ public class ReplicationControlManagerTest { .setResourceName(topic) .setName(configKey) .setValue(configValue); - replay(singletonList(new ApiMessageAndVersion(configRecord, (short) 0))); + replay(List.of(new ApiMessageAndVersion(configRecord, (short) 0))); } void fenceBrokers(Integer... brokerIds) { @@ -633,7 +629,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData(); expectedResponse.topics().add(new CreatableTopicResult().setName("foo"). setErrorCode(INVALID_REPLICATION_FACTOR.code()). @@ -646,7 +642,7 @@ public class ReplicationControlManagerTest { ctx.inControlledShutdownBrokers(0); ControllerResult result2 = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse2 = new CreateTopicsResponseData(); expectedResponse2.topics().add(new CreatableTopicResult().setName("foo"). setErrorCode(INVALID_REPLICATION_FACTOR.code()). @@ -658,7 +654,7 @@ public class ReplicationControlManagerTest { ctx.unfenceBrokers(0, 1, 2); ControllerResult result3 = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse3 = new CreateTopicsResponseData(); expectedResponse3.topics().add(new CreatableTopicResult().setName("foo"). setNumPartitions(1).setReplicationFactor((short) 3). @@ -676,7 +672,7 @@ public class ReplicationControlManagerTest { replicationControl.getPartition( ((TopicRecord) result3.records().get(0).message()).topicId(), 0)); ControllerResult result4 = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse4 = new CreateTopicsResponseData(); expectedResponse4.topics().add(new CreatableTopicResult().setName("foo"). setErrorCode(Errors.TOPIC_ALREADY_EXISTS.code()). @@ -696,7 +692,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextWithMutationQuotaExceededFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData(); expectedResponse.topics().add(new CreatableTopicResult().setName("foo"). setErrorCode(THROTTLING_QUOTA_EXCEEDED.code()). @@ -719,7 +715,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData(); expectedResponse.topics().add(new CreatableTopicResult().setName("foo"). @@ -772,7 +768,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result1 = - replicationControl.createTopics(requestContext, request1, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request1, Set.of("foo")); assertEquals((short) 0, result1.response().topics().find("foo").errorCode()); List records1 = result1.records(); @@ -805,7 +801,7 @@ public class ReplicationControlManagerTest { .setConfigs(invalidConfigs)); ControllerResult result2 = - replicationControl.createTopics(requestContext, request2, Collections.singleton("bar")); + replicationControl.createTopics(requestContext, request2, Set.of("bar")); assertEquals(Errors.INVALID_CONFIG.code(), result2.response().topics().find("bar").errorCode()); assertEquals( "Null value not supported for topic configs: foo", @@ -818,9 +814,9 @@ public class ReplicationControlManagerTest { .setConfigs(validConfigs)); ControllerResult result3 = - replicationControl.createTopics(requestContext, request3, Collections.singleton("baz")); + replicationControl.createTopics(requestContext, request3, Set.of("baz")); assertEquals(INVALID_REPLICATION_FACTOR.code(), result3.response().topics().find("baz").errorCode()); - assertEquals(Collections.emptyList(), result3.records()); + assertEquals(List.of(), result3.records()); // Test request with multiple topics together. CreateTopicsRequestData request4 = new CreateTopicsRequestData(); @@ -869,7 +865,7 @@ public class ReplicationControlManagerTest { anonymousContextWithMutationQuotaExceededFor(ApiKeys.CREATE_TOPICS) : anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - ctx.replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + ctx.replicationControl.createTopics(requestContext, request, Set.of("foo")); assertEquals(0, result.records().size()); CreatableTopicResult topicResult = result.response().topics().find("foo"); if (mutationQuotaExceeded) { @@ -889,7 +885,7 @@ public class ReplicationControlManagerTest { setNumPartitions(1).setReplicationFactor((short) 4)); ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - ctx.replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + ctx.replicationControl.createTopics(requestContext, request, Set.of("foo")); assertEquals(0, result.records().size()); CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData(); expectedResponse.topics().add(new CreatableTopicResult().setName("foo"). @@ -902,16 +898,16 @@ public class ReplicationControlManagerTest { @Test public void testCreateTopicsWithPolicy() { - MockCreateTopicPolicy createTopicPolicy = new MockCreateTopicPolicy(asList( + MockCreateTopicPolicy createTopicPolicy = new MockCreateTopicPolicy(List.of( new CreateTopicPolicy.RequestMetadata("foo", 2, (short) 2, - null, Collections.emptyMap()), + null, Map.of()), new CreateTopicPolicy.RequestMetadata("bar", 3, (short) 2, - null, Collections.emptyMap()), + null, Map.of()), new CreateTopicPolicy.RequestMetadata("baz", null, null, - Collections.singletonMap(0, asList(2, 1, 0)), - Collections.singletonMap(SEGMENT_BYTES_CONFIG, "12300000")), + Map.of(0, List.of(2, 1, 0)), + Map.of(SEGMENT_BYTES_CONFIG, "12300000")), new CreateTopicPolicy.RequestMetadata("quux", null, null, - Collections.singletonMap(0, asList(2, 1, 0)), Collections.emptyMap()))); + Map.of(0, List.of(2, 1, 0)), Map.of()))); ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder(). setCreateTopicPolicy(createTopicPolicy). build(); @@ -920,7 +916,7 @@ public class ReplicationControlManagerTest { ctx.createTestTopic("foo", 2, (short) 2, NONE.code()); ctx.createTestTopic("bar", 3, (short) 3, POLICY_VIOLATION.code()); ctx.createTestTopic("baz", new int[][] {new int[] {2, 1, 0}}, - Collections.singletonMap(SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); + Map.of(SEGMENT_BYTES_CONFIG, "12300000"), NONE.code()); ctx.createTestTopic("quux", new int[][] {new int[] {1, 2, 0}}, POLICY_VIOLATION.code()); } @@ -947,7 +943,7 @@ public class ReplicationControlManagerTest { topics.add(new CreatableTopic().setName("")); topics.add(new CreatableTopic().setName("woo")); topics.add(new CreatableTopic().setName(".")); - ReplicationControlManager.validateNewTopicNames(topicErrors, topics, Collections.emptyMap()); + ReplicationControlManager.validateNewTopicNames(topicErrors, topics, Map.of()); Map expectedTopicErrors = new HashMap<>(); expectedTopicErrors.put("", new ApiError(INVALID_TOPIC_EXCEPTION, "Topic name is invalid: the empty string is not allowed")); @@ -963,8 +959,8 @@ public class ReplicationControlManagerTest { topics.add(new CreatableTopic().setName("foo.bar")); topics.add(new CreatableTopic().setName("woo.bar_foo")); Map> collisionMap = new HashMap<>(); - collisionMap.put("foo_bar", new TreeSet<>(singletonList("foo_bar"))); - collisionMap.put("woo_bar_foo", new TreeSet<>(Arrays.asList("woo.bar.foo", "woo_bar.foo"))); + collisionMap.put("foo_bar", new TreeSet<>(List.of("foo_bar"))); + collisionMap.put("woo_bar_foo", new TreeSet<>(List.of("woo.bar.foo", "woo_bar.foo"))); ReplicationControlManager.validateNewTopicNames(topicErrors, topics, collisionMap); Map expectedTopicErrors = new HashMap<>(); expectedTopicErrors.put("foo.bar", new ApiError(INVALID_TOPIC_EXCEPTION, @@ -995,7 +991,7 @@ public class ReplicationControlManagerTest { List records = new ArrayList<>(); replicationControl.handleBrokerFenced(0, records); ctx.replay(records); - assertEquals(Collections.emptySet(), RecordTestUtils. + assertEquals(Set.of(), RecordTestUtils. iteratorToSet(replicationControl.brokersToIsrs().iterator(0, true))); } @@ -1254,9 +1250,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData request = new AlterPartitionRequestData() .setBrokerId(0) .setBrokerEpoch(100) - .setTopics(singletonList(new TopicData() + .setTopics(List.of(new TopicData() .setTopicId(topicId) - .setPartitions(singletonList(new PartitionData() + .setPartitions(List.of(new PartitionData() .setPartitionIndex(0))))); ControllerRequestContext requestContext = @@ -1267,9 +1263,9 @@ public class ReplicationControlManagerTest { Errors expectedError = UNKNOWN_TOPIC_ID; AlterPartitionResponseData expectedResponse = new AlterPartitionResponseData() - .setTopics(singletonList(new AlterPartitionResponseData.TopicData() + .setTopics(List.of(new AlterPartitionResponseData.TopicData() .setTopicId(topicId) - .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() + .setPartitions(List.of(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))); @@ -1446,7 +1442,7 @@ public class ReplicationControlManagerTest { ) { Map configs = ctx.configurationControl.getConfigs( new ConfigResource(ConfigResource.Type.TOPIC, topic)); - assertEquals(Collections.emptyMap(), configs); + assertEquals(Map.of(), configs); } @Test @@ -1467,7 +1463,7 @@ public class ReplicationControlManagerTest { ctx.unfenceBrokers(0, 1); ControllerRequestContext createTopicsRequestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult createResult = - replicationControl.createTopics(createTopicsRequestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(createTopicsRequestContext, request, Set.of("foo")); CreateTopicsResponseData expectedResponse = new CreateTopicsResponseData(); Uuid topicId = createResult.response().topics().find("foo").topicId(); expectedResponse.topics().add(new CreatableTopicResult().setName("foo"). @@ -1485,29 +1481,29 @@ public class ReplicationControlManagerTest { assertNull(replicationControl.getPartition(topicId, 3)); assertCreatedTopicConfigs(ctx, "foo", requestConfigs); - assertEquals(singletonMap(topicId, new ResultOrError<>("foo")), - replicationControl.findTopicNames(Long.MAX_VALUE, Collections.singleton(topicId))); - assertEquals(singletonMap("foo", new ResultOrError<>(topicId)), - replicationControl.findTopicIds(Long.MAX_VALUE, Collections.singleton("foo"))); + assertEquals(Map.of(topicId, new ResultOrError<>("foo")), + replicationControl.findTopicNames(Long.MAX_VALUE, Set.of(topicId))); + assertEquals(Map.of("foo", new ResultOrError<>(topicId)), + replicationControl.findTopicIds(Long.MAX_VALUE, Set.of("foo"))); Uuid invalidId = new Uuid(topicId.getMostSignificantBits() + 1, topicId.getLeastSignificantBits()); - assertEquals(singletonMap(invalidId, + assertEquals(Map.of(invalidId, new ResultOrError<>(new ApiError(UNKNOWN_TOPIC_ID))), - replicationControl.findTopicNames(Long.MAX_VALUE, Collections.singleton(invalidId))); - assertEquals(singletonMap("bar", + replicationControl.findTopicNames(Long.MAX_VALUE, Set.of(invalidId))); + assertEquals(Map.of("bar", new ResultOrError<>(new ApiError(UNKNOWN_TOPIC_OR_PARTITION))), - replicationControl.findTopicIds(Long.MAX_VALUE, Collections.singleton("bar"))); + replicationControl.findTopicIds(Long.MAX_VALUE, Set.of("bar"))); ControllerRequestContext deleteTopicsRequestContext = anonymousContextFor(ApiKeys.DELETE_TOPICS); ControllerResult> invalidDeleteResult = replicationControl. - deleteTopics(deleteTopicsRequestContext, Collections.singletonList(invalidId)); + deleteTopics(deleteTopicsRequestContext, List.of(invalidId)); assertEquals(0, invalidDeleteResult.records().size()); - assertEquals(singletonMap(invalidId, new ApiError(UNKNOWN_TOPIC_ID, null)), + assertEquals(Map.of(invalidId, new ApiError(UNKNOWN_TOPIC_ID, null)), invalidDeleteResult.response()); ControllerResult> deleteResult = replicationControl. - deleteTopics(deleteTopicsRequestContext, Collections.singletonList(topicId)); + deleteTopics(deleteTopicsRequestContext, List.of(topicId)); assertTrue(deleteResult.isAtomic()); - assertEquals(singletonMap(topicId, new ApiError(NONE, null)), + assertEquals(Map.of(topicId, new ApiError(NONE, null)), deleteResult.response()); assertEquals(1, deleteResult.records().size()); ctx.replay(deleteResult.records()); @@ -1515,12 +1511,12 @@ public class ReplicationControlManagerTest { assertNull(replicationControl.getPartition(topicId, 1)); assertNull(replicationControl.getPartition(topicId, 2)); assertNull(replicationControl.getPartition(topicId, 3)); - assertEquals(singletonMap(topicId, new ResultOrError<>( + assertEquals(Map.of(topicId, new ResultOrError<>( new ApiError(UNKNOWN_TOPIC_ID))), replicationControl.findTopicNames( - Long.MAX_VALUE, Collections.singleton(topicId))); - assertEquals(singletonMap("foo", new ResultOrError<>( + Long.MAX_VALUE, Set.of(topicId))); + assertEquals(Map.of("foo", new ResultOrError<>( new ApiError(UNKNOWN_TOPIC_OR_PARTITION))), replicationControl.findTopicIds( - Long.MAX_VALUE, Collections.singleton("foo"))); + Long.MAX_VALUE, Set.of("foo"))); assertEmptyTopicConfigs(ctx, "foo"); } @@ -1536,7 +1532,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext createTopicsRequestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult createResult = - replicationControl.createTopics(createTopicsRequestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(createTopicsRequestContext, request, Set.of("foo")); CreatableTopicResult createdTopic = createResult.response().topics().find("foo"); assertEquals(NONE.code(), createdTopic.errorCode()); ctx.replay(createResult.records()); @@ -1544,8 +1540,8 @@ public class ReplicationControlManagerTest { anonymousContextWithMutationQuotaExceededFor(ApiKeys.DELETE_TOPICS); Uuid topicId = createdTopic.topicId(); ControllerResult> deleteResult = replicationControl. - deleteTopics(deleteTopicsRequestContext, Collections.singletonList(topicId)); - assertEquals(singletonMap(topicId, new ApiError(THROTTLING_QUOTA_EXCEEDED, QUOTA_EXCEEDED_IN_TEST_MSG)), + deleteTopics(deleteTopicsRequestContext, List.of(topicId)); + assertEquals(Map.of(topicId, new ApiError(THROTTLING_QUOTA_EXCEEDED, QUOTA_EXCEEDED_IN_TEST_MSG)), deleteResult.response()); assertEquals(0, deleteResult.records().size()); } @@ -1564,13 +1560,13 @@ public class ReplicationControlManagerTest { request.topics().add(new CreatableTopic().setName("foo2"). setNumPartitions(2).setReplicationFactor((short) 2)); ctx.registerBrokersWithDirs( - 0, Collections.emptyList(), - 1, asList(Uuid.fromString("QMzamNQVQ7GnJK9DwQHG7Q"), Uuid.fromString("loDxEBLETdedNnQGOKKENw")), - 3, Collections.singletonList(Uuid.fromString("dxCDSgNjQvS4WuyqEKoCwA"))); + 0, List.of(), + 1, List.of(Uuid.fromString("QMzamNQVQ7GnJK9DwQHG7Q"), Uuid.fromString("loDxEBLETdedNnQGOKKENw")), + 3, List.of(Uuid.fromString("dxCDSgNjQvS4WuyqEKoCwA"))); ctx.unfenceBrokers(0, 1, 3); ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult createTopicResult = replicationControl. - createTopics(requestContext, request, new HashSet<>(Arrays.asList("foo", "bar", "quux", "foo2"))); + createTopics(requestContext, request, new HashSet<>(List.of("foo", "bar", "quux", "foo2"))); ctx.replay(createTopicResult.records()); List topics = new ArrayList<>(); topics.add(new CreatePartitionsTopic(). @@ -1583,7 +1579,7 @@ public class ReplicationControlManagerTest { setName("quux").setCount(2).setAssignments(null)); ControllerResult> createPartitionsResult = replicationControl.createPartitions(requestContext, topics); - assertEquals(asList(new CreatePartitionsTopicResult(). + assertEquals(List.of(new CreatePartitionsTopicResult(). setName("foo"). setErrorCode(NONE.code()). setErrorMessage(null), @@ -1603,20 +1599,20 @@ public class ReplicationControlManagerTest { ctx.replay(createPartitionsResult.records()); List topics2 = new ArrayList<>(); topics2.add(new CreatePartitionsTopic(). - setName("foo").setCount(6).setAssignments(singletonList( - new CreatePartitionsAssignment().setBrokerIds(asList(1, 3))))); + setName("foo").setCount(6).setAssignments(List.of( + new CreatePartitionsAssignment().setBrokerIds(List.of(1, 3))))); topics2.add(new CreatePartitionsTopic(). - setName("bar").setCount(5).setAssignments(singletonList( - new CreatePartitionsAssignment().setBrokerIds(singletonList(1))))); + setName("bar").setCount(5).setAssignments(List.of( + new CreatePartitionsAssignment().setBrokerIds(List.of(1))))); topics2.add(new CreatePartitionsTopic(). - setName("quux").setCount(4).setAssignments(singletonList( - new CreatePartitionsAssignment().setBrokerIds(asList(1, 0))))); + setName("quux").setCount(4).setAssignments(List.of( + new CreatePartitionsAssignment().setBrokerIds(List.of(1, 0))))); topics2.add(new CreatePartitionsTopic(). - setName("foo2").setCount(3).setAssignments(singletonList( - new CreatePartitionsAssignment().setBrokerIds(asList(2, 0))))); + setName("foo2").setCount(3).setAssignments(List.of( + new CreatePartitionsAssignment().setBrokerIds(List.of(2, 0))))); ControllerResult> createPartitionsResult2 = replicationControl.createPartitions(requestContext, topics2); - assertEquals(asList(new CreatePartitionsTopicResult(). + assertEquals(List.of(new CreatePartitionsTopicResult(). setName("foo"). setErrorCode(NONE.code()). setErrorMessage(null), @@ -1654,7 +1650,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext createTopicsRequestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult createResult = - replicationControl.createTopics(createTopicsRequestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(createTopicsRequestContext, request, Set.of("foo")); CreatableTopicResult createdTopic = createResult.response().topics().find("foo"); assertEquals(NONE.code(), createdTopic.errorCode()); ctx.replay(createResult.records()); @@ -1665,7 +1661,7 @@ public class ReplicationControlManagerTest { anonymousContextWithMutationQuotaExceededFor(ApiKeys.CREATE_PARTITIONS); ControllerResult> createPartitionsResult = replicationControl.createPartitions(createPartitionsRequestContext, topics); - List expectedThrottled = singletonList(new CreatePartitionsTopicResult(). + List expectedThrottled = List.of(new CreatePartitionsTopicResult(). setName("foo"). setErrorCode(THROTTLING_QUOTA_EXCEEDED.code()). setErrorMessage(QUOTA_EXCEEDED_IN_TEST_MSG)); @@ -1673,8 +1669,8 @@ public class ReplicationControlManagerTest { // now test the explicit assignment case List topics2 = new ArrayList<>(); topics2.add(new CreatePartitionsTopic(). - setName("foo").setCount(4).setAssignments(singletonList( - new CreatePartitionsAssignment().setBrokerIds(asList(1, 0))))); + setName("foo").setCount(4).setAssignments(List.of( + new CreatePartitionsAssignment().setBrokerIds(List.of(1, 0))))); ControllerResult> createPartitionsResult2 = replicationControl.createPartitions(createPartitionsRequestContext, topics2); assertEquals(expectedThrottled, createPartitionsResult2.response()); @@ -1694,7 +1690,7 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult createTopicResult = replicationControl. - createTopics(requestContext, request, new HashSet<>(singletonList("foo"))); + createTopics(requestContext, request, new HashSet<>(List.of("foo"))); ctx.replay(createTopicResult.records()); ctx.registerBrokers(0, 1); @@ -1708,7 +1704,7 @@ public class ReplicationControlManagerTest { replicationControl.createPartitions(requestContext, topics); assertEquals( - singletonList(new CreatePartitionsTopicResult(). + List.of(new CreatePartitionsTopicResult(). setName("foo"). setErrorCode(INVALID_REPLICATION_FACTOR.code()). setErrorMessage("Unable to replicate the partition 2 time(s): All " + @@ -1731,10 +1727,10 @@ public class ReplicationControlManagerTest { ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS); ControllerResult result = - replicationControl.createTopics(requestContext, request, Collections.singleton("foo")); + replicationControl.createTopics(requestContext, request, Set.of("foo")); ctx.replay(result.records()); - List topics = singletonList(new CreatePartitionsTopic(). + List topics = List.of(new CreatePartitionsTopic(). setName("foo").setCount(2).setAssignments(null)); ControllerResult> createPartitionsResult = @@ -1764,13 +1760,13 @@ public class ReplicationControlManagerTest { public void testValidateGoodManualPartitionAssignments() { ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().build(); ctx.registerBrokers(1, 2, 3); - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(singletonList(1)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1)), OptionalInt.of(1)); - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(singletonList(1)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1)), OptionalInt.empty()); - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1, 2, 3)), OptionalInt.of(3)); - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1, 2, 3)), OptionalInt.empty()); } @@ -1780,20 +1776,20 @@ public class ReplicationControlManagerTest { ctx.registerBrokers(1, 2); assertEquals("The manual partition assignment includes an empty replica list.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(Collections.emptyList()), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of()), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes broker 3, but no such " + "broker is registered.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 3)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1, 2, 3)), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes the broker 2 more than " + "once.", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2, 2)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1, 2, 2)), OptionalInt.empty())).getMessage()); assertEquals("The manual partition assignment includes a partition with 2 " + "replica(s), but this is not consistent with previous partitions, which have " + "3 replica(s).", assertThrows(InvalidReplicaAssignmentException.class, () -> - ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(asList(1, 2)), + ctx.replicationControl.validateManualPartitionAssignment(partitionAssignment(List.of(1, 2)), OptionalInt.of(3))).getMessage()); } @@ -1817,18 +1813,18 @@ public class ReplicationControlManagerTest { assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(null, Long.MAX_VALUE)); ControllerResult alterResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(asList( - new ReassignableTopic().setName("foo").setPartitions(asList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(3, 2, 1)), + setReplicas(List.of(3, 2, 1)), new ReassignablePartition().setPartitionIndex(1). - setReplicas(asList(0, 2, 1)), + setReplicas(List.of(0, 2, 1)), new ReassignablePartition().setPartitionIndex(2). - setReplicas(asList(0, 2, 1)))), + setReplicas(List.of(0, 2, 1)))), new ReassignableTopic().setName("bar")))); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setResponses(asList( - new ReassignableTopicResponse().setName("foo").setPartitions(asList( + setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null), new ReassignablePartitionResponse().setPartitionIndex(1). @@ -1842,46 +1838,46 @@ public class ReplicationControlManagerTest { ctx.replay(alterResult.records()); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName("foo").setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName("foo").setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(1). - setRemovingReplicas(singletonList(3)). - setAddingReplicas(singletonList(0)). - setReplicas(asList(0, 2, 1, 3)))))); + setRemovingReplicas(List.of(3)). + setAddingReplicas(List.of(0)). + setReplicas(List.of(0, 2, 1, 3)))))); assertEquals(currentReassigning, replication.listPartitionReassignments(null, Long.MAX_VALUE)); - assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(singletonList( + assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("bar"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); - assertEquals(currentReassigning, replication.listPartitionReassignments(singletonList( + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); + assertEquals(currentReassigning, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("foo"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); ControllerResult cancelResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(asList( - new ReassignableTopic().setName("foo").setPartitions(asList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). setReplicas(null), new ReassignablePartition().setPartitionIndex(1). setReplicas(null), new ReassignablePartition().setPartitionIndex(2). setReplicas(null))), - new ReassignableTopic().setName("bar").setPartitions(singletonList( + new ReassignableTopic().setName("bar").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). setReplicas(null)))))); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(fooId). setPartitionId(1). - setReplicas(asList(2, 1, 3)). - setDirectories(asList( + setReplicas(List.of(2, 1, 3)). + setDirectories(List.of( Uuid.fromString("TESTBROKER00002DIRAAAA"), Uuid.fromString("TESTBROKER00001DIRAAAA"), Uuid.fromString("TESTBROKER00003DIRAAAA") )). setLeader(3). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()), MetadataVersion.latestTesting().partitionChangeRecordVersion())), - new AlterPartitionReassignmentsResponseData().setErrorMessage(null).setResponses(asList( - new ReassignableTopicResponse().setName("foo").setPartitions(asList( + setRemovingReplicas(List.of()). + setAddingReplicas(List.of()), MetadataVersion.latestTesting().partitionChangeRecordVersion())), + new AlterPartitionReassignmentsResponseData().setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorCode(NO_REASSIGNMENT_IN_PROGRESS.code()).setErrorMessage(null), new ReassignablePartitionResponse().setPartitionIndex(1). @@ -1889,7 +1885,7 @@ public class ReplicationControlManagerTest { new ReassignablePartitionResponse().setPartitionIndex(2). setErrorCode(UNKNOWN_TOPIC_OR_PARTITION.code()). setErrorMessage("Unable to find partition foo:2."))), - new ReassignableTopicResponse().setName("bar").setPartitions(singletonList( + new ReassignableTopicResponse().setName("bar").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorCode(NO_REASSIGNMENT_IN_PROGRESS.code()). setErrorMessage(null)))))), @@ -1900,9 +1896,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterPartitionRequestData = new AlterPartitionRequestData(). setBrokerId(3). setBrokerEpoch(103). - setTopics(singletonList(new TopicData(). + setTopics(List.of(new TopicData(). setTopicId(fooId). - setPartitions(singletonList(new PartitionData(). + setPartitions(List.of(new PartitionData(). setPartitionIndex(1). setPartitionEpoch(1). setLeaderEpoch(0). @@ -1911,10 +1907,10 @@ public class ReplicationControlManagerTest { requestContext, new AlterPartitionRequest.Builder(alterPartitionRequestData).build(version).data()); Errors expectedError = NEW_LEADER_ELECTED; - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( new AlterPartitionResponseData.TopicData(). setTopicId(fooId). - setPartitions(singletonList( + setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(1). setErrorCode(expectedError.code()))))), @@ -1937,18 +1933,18 @@ public class ReplicationControlManagerTest { ControllerResult alterResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName("foo").setPartitions(asList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(1, 2, 3)), + setReplicas(List.of(1, 2, 3)), new ReassignablePartition().setPartitionIndex(1). - setReplicas(asList(0, 1)), + setReplicas(List.of(0, 1)), new ReassignablePartition().setPartitionIndex(2). - setReplicas(asList(0, 1, 2, 3)))))). + setReplicas(List.of(0, 1, 2, 3)))))). setAllowReplicationFactorChange(false)); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(singletonList( - new ReassignableTopicResponse().setName("foo").setPartitions(asList( + setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null), new ReassignablePartitionResponse().setPartitionIndex(1). @@ -1961,26 +1957,26 @@ public class ReplicationControlManagerTest { ctx.replay(alterResult.records()); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName("foo").setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName("foo").setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(0). - setRemovingReplicas(singletonList(0)). - setAddingReplicas(singletonList(3)). - setReplicas(asList(1, 2, 3, 0)))))); - assertEquals(currentReassigning, replication.listPartitionReassignments(singletonList( + setRemovingReplicas(List.of(0)). + setAddingReplicas(List.of(3)). + setReplicas(List.of(1, 2, 3, 0)))))); + assertEquals(currentReassigning, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("foo"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); // test alter replica factor not allow to change when partition reassignment is ongoing ControllerResult alterReassigningResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName("foo").setPartitions(singletonList( - new ReassignablePartition().setPartitionIndex(0).setReplicas(asList(0, 1)))))). + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( + new ReassignablePartition().setPartitionIndex(0).setReplicas(List.of(0, 1)))))). setAllowReplicationFactorChange(false)); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(singletonList( - new ReassignableTopicResponse().setName("foo").setPartitions(singletonList( + setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorCode(INVALID_REPLICATION_FACTOR.code()). setErrorMessage("The replication factor is changed from 3 to 2"))))), @@ -1988,13 +1984,13 @@ public class ReplicationControlManagerTest { ControllerResult alterReassigningResult2 = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName("foo").setPartitions(singletonList( - new ReassignablePartition().setPartitionIndex(0).setReplicas(asList(0, 2, 3)))))). + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( + new ReassignablePartition().setPartitionIndex(0).setReplicas(List.of(0, 2, 3)))))). setAllowReplicationFactorChange(false)); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(singletonList( - new ReassignableTopicResponse().setName("foo").setPartitions(singletonList( + setErrorMessage(null).setAllowReplicationFactorChange(false).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null))))), alterReassigningResult2.response()); @@ -2014,38 +2010,38 @@ public class ReplicationControlManagerTest { ControllerResult alterResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName("foo").setPartitions(singletonList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(1, 2, 3))))))); + setReplicas(List.of(1, 2, 3))))))); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setResponses(singletonList( - new ReassignableTopicResponse().setName("foo").setPartitions(singletonList( + setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0).setErrorMessage(null))))), alterResult.response()); ctx.replay(alterResult.records()); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName("foo").setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName("foo").setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(0). - setRemovingReplicas(singletonList(0)). - setAddingReplicas(singletonList(3)). - setReplicas(asList(1, 2, 3, 0)))))); - assertEquals(currentReassigning, replication.listPartitionReassignments(singletonList( + setRemovingReplicas(List.of(0)). + setAddingReplicas(List.of(3)). + setReplicas(List.of(1, 2, 3, 0)))))); + assertEquals(currentReassigning, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("foo"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); // test replica factor change check takes no effect when partition reassignment is ongoing ControllerResult cancelResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName("foo").setPartitions(singletonList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0).setReplicas(null))))). setAllowReplicationFactorChange(false)); assertEquals(new AlterPartitionReassignmentsResponseData().setAllowReplicationFactorChange(false).setErrorMessage(null). - setResponses(singletonList( - new ReassignableTopicResponse().setName("foo").setPartitions(singletonList( + setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0).setErrorMessage(null))))), cancelResult.response()); ctx.replay(cancelResult.records()); @@ -2088,9 +2084,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterIsrRequest = new AlterPartitionRequestData() .setBrokerId(1) .setBrokerEpoch(101) - .setTopics(singletonList(new TopicData() + .setTopics(List.of(new TopicData() .setTopicId(fooId) - .setPartitions(singletonList(new PartitionData() + .setPartitions(List.of(new PartitionData() .setPartitionIndex(0) .setPartitionEpoch(1) .setLeaderEpoch(0) @@ -2105,9 +2101,9 @@ public class ReplicationControlManagerTest { Errors expectedError = INELIGIBLE_REPLICA; assertEquals( new AlterPartitionResponseData() - .setTopics(singletonList(new AlterPartitionResponseData.TopicData() + .setTopics(List.of(new AlterPartitionResponseData.TopicData() .setTopicId(fooId) - .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() + .setPartitions(List.of(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))), alterPartitionResult.response()); @@ -2120,13 +2116,13 @@ public class ReplicationControlManagerTest { assertEquals( new AlterPartitionResponseData() - .setTopics(singletonList(new AlterPartitionResponseData.TopicData() + .setTopics(List.of(new AlterPartitionResponseData.TopicData() .setTopicId(fooId) - .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() + .setPartitions(List.of(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setLeaderId(1) .setLeaderEpoch(0) - .setIsr(asList(1, 2, 3, 4)) + .setIsr(List.of(1, 2, 3, 4)) .setPartitionEpoch(2) .setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -2149,9 +2145,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterIsrRequest = new AlterPartitionRequestData(). setBrokerId(1). setBrokerEpoch(101). - setTopics(singletonList(new TopicData(). + setTopics(List.of(new TopicData(). setTopicId(fooId). - setPartitions(singletonList(new PartitionData(). + setPartitions(List.of(new PartitionData(). setPartitionIndex(0). setPartitionEpoch(1). setLeaderEpoch(0). @@ -2166,7 +2162,7 @@ public class ReplicationControlManagerTest { setPort((short) 9092 + 4). setName("PLAINTEXT"). setHost("localhost")); - ctx.replay(Collections.singletonList(new ApiMessageAndVersion(brokerRecord, (short) 0))); + ctx.replay(List.of(new ApiMessageAndVersion(brokerRecord, (short) 0))); // Unfence the broker 4. ControllerResult result = ctx.replicationControl. @@ -2188,9 +2184,9 @@ public class ReplicationControlManagerTest { if (version >= 3) { assertEquals( new AlterPartitionResponseData(). - setTopics(singletonList(new AlterPartitionResponseData.TopicData(). + setTopics(List.of(new AlterPartitionResponseData.TopicData(). setTopicId(fooId). - setPartitions(singletonList(new AlterPartitionResponseData.PartitionData(). + setPartitions(List.of(new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setErrorCode(INELIGIBLE_REPLICA.code()))))), alterPartitionResult.response()); @@ -2233,9 +2229,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterIsrRequest = new AlterPartitionRequestData() .setBrokerId(1) .setBrokerEpoch(101) - .setTopics(singletonList(new TopicData() + .setTopics(List.of(new TopicData() .setTopicId(fooId) - .setPartitions(singletonList(new PartitionData() + .setPartitions(List.of(new PartitionData() .setPartitionIndex(0) .setPartitionEpoch(0) .setLeaderEpoch(0) @@ -2250,9 +2246,9 @@ public class ReplicationControlManagerTest { Errors expectedError = INELIGIBLE_REPLICA; assertEquals( new AlterPartitionResponseData() - .setTopics(singletonList(new AlterPartitionResponseData.TopicData() + .setTopics(List.of(new AlterPartitionResponseData.TopicData() .setTopicId(fooId) - .setPartitions(singletonList(new AlterPartitionResponseData.PartitionData() + .setPartitions(List.of(new AlterPartitionResponseData.PartitionData() .setPartitionIndex(0) .setErrorCode(expectedError.code()))))), alterPartitionResult.response()); @@ -2286,30 +2282,30 @@ public class ReplicationControlManagerTest { setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(0).setPartitionEpoch(1).build(), replication.getPartition(fooId, 0)); ControllerResult alterResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(asList( - new ReassignableTopic().setName("foo").setPartitions(asList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(1, 2, 4)), + setReplicas(List.of(1, 2, 4)), new ReassignablePartition().setPartitionIndex(1). - setReplicas(asList(1, 2, 3, 0)), + setReplicas(List.of(1, 2, 3, 0)), new ReassignablePartition().setPartitionIndex(2). - setReplicas(asList(5, 6, 7)), + setReplicas(List.of(5, 6, 7)), new ReassignablePartition().setPartitionIndex(3). - setReplicas(Collections.emptyList()))), - new ReassignableTopic().setName("bar").setPartitions(singletonList( + setReplicas(List.of()))), + new ReassignableTopic().setName("bar").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(1, 2, 3, 4, 0))))))); + setReplicas(List.of(1, 2, 3, 4, 0))))))); assertEquals(new AlterPartitionReassignmentsResponseData(). setErrorMessage(null). - setResponses(asList( - new ReassignableTopicResponse().setName("foo").setPartitions(asList( + setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0).setErrorMessage(null), new ReassignablePartitionResponse().setPartitionIndex(1).setErrorMessage(null), new ReassignablePartitionResponse().setPartitionIndex(2).setErrorCode(INVALID_REPLICA_ASSIGNMENT.code()). setErrorMessage("The manual partition assignment includes broker 5, but no such broker is registered."), new ReassignablePartitionResponse().setPartitionIndex(3).setErrorCode(INVALID_REPLICA_ASSIGNMENT.code()). setErrorMessage("The manual partition assignment includes an empty replica list."))), - new ReassignableTopicResponse().setName("bar").setPartitions(singletonList( + new ReassignableTopicResponse().setName("bar").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0).setErrorMessage(null))))), alterResult.response()); ctx.replay(alterResult.records()); @@ -2339,61 +2335,61 @@ public class ReplicationControlManagerTest { setAddingReplicas(new int[] {0, 1}).setLeader(4).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(0).setPartitionEpoch(2).build(), replication.getPartition(barId, 0)); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName("bar").setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName("bar").setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(0). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(asList(0, 1)). - setReplicas(asList(1, 2, 3, 4, 0)))))); + setRemovingReplicas(List.of()). + setAddingReplicas(List.of(0, 1)). + setReplicas(List.of(1, 2, 3, 4, 0)))))); assertEquals(currentReassigning, replication.listPartitionReassignments(null, Long.MAX_VALUE)); - assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(singletonList( + assertEquals(NONE_REASSIGNING, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("foo"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); - assertEquals(currentReassigning, replication.listPartitionReassignments(singletonList( + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); + assertEquals(currentReassigning, replication.listPartitionReassignments(List.of( new ListPartitionReassignmentsTopics().setName("bar"). - setPartitionIndexes(asList(0, 1, 2))), Long.MAX_VALUE)); + setPartitionIndexes(List.of(0, 1, 2))), Long.MAX_VALUE)); ControllerResult alterPartitionResult = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(4).setBrokerEpoch(104). - setTopics(singletonList(new TopicData().setTopicId(barId).setPartitions(singletonList( + setTopics(List.of(new TopicData().setTopicId(barId).setPartitions(List.of( new PartitionData().setPartitionIndex(0).setPartitionEpoch(2). setLeaderEpoch(0).setNewIsrWithEpochs(isrWithDefaultEpoch(4, 1, 2, 0))))))); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( - new AlterPartitionResponseData.TopicData().setTopicId(barId).setPartitions(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( + new AlterPartitionResponseData.TopicData().setTopicId(barId).setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(4). setLeaderEpoch(0). - setIsr(asList(4, 1, 2, 0)). + setIsr(List.of(4, 1, 2, 0)). setPartitionEpoch(3). setErrorCode(NONE.code()))))), alterPartitionResult.response()); ControllerResult cancelResult = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(asList( - new ReassignableTopic().setName("foo").setPartitions(singletonList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName("foo").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). setReplicas(null))), - new ReassignableTopic().setName("bar").setPartitions(singletonList( + new ReassignableTopic().setName("bar").setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). setReplicas(null)))))); - assertEquals(ControllerResult.atomicOf(Collections.singletonList(new ApiMessageAndVersion( + assertEquals(ControllerResult.atomicOf(List.of(new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(barId). setPartitionId(0). setLeader(4). - setReplicas(asList(2, 3, 4)). - setDirectories(asList( + setReplicas(List.of(2, 3, 4)). + setDirectories(List.of( Uuid.fromString("TESTBROKER00002DIRAAAA"), Uuid.fromString("TESTBROKER00003DIRAAAA"), Uuid.fromString("TESTBROKER00004DIRAAAA") )). setRemovingReplicas(null). - setAddingReplicas(Collections.emptyList()), MetadataVersion.latestTesting().partitionChangeRecordVersion())), - new AlterPartitionReassignmentsResponseData().setErrorMessage(null).setResponses(asList( - new ReassignableTopicResponse().setName("foo").setPartitions(singletonList( + setAddingReplicas(List.of()), MetadataVersion.latestTesting().partitionChangeRecordVersion())), + new AlterPartitionReassignmentsResponseData().setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName("foo").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorCode(NO_REASSIGNMENT_IN_PROGRESS.code()).setErrorMessage(null))), - new ReassignableTopicResponse().setName("bar").setPartitions(singletonList( + new ReassignableTopicResponse().setName("bar").setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null)))))), cancelResult); @@ -2475,12 +2471,12 @@ public class ReplicationControlManagerTest { ElectLeadersRequestData request = buildElectLeadersRequest( ElectionType.UNCLEAN, - electAllPartitions ? null : singletonMap("foo", asList(0, 1, 2)) + electAllPartitions ? null : Map.of("foo", List.of(0, 1, 2)) ); // No election can be done yet because no replicas are available for partition 0 ControllerResult result1 = replication.electLeaders(request); - assertEquals(Collections.emptyList(), result1.records()); + assertEquals(List.of(), result1.records()); ElectLeadersResponseData expectedResponse1 = buildElectLeadersResponse(NONE, electAllPartitions, Utils.mkMap( Utils.mkEntry( @@ -2514,7 +2510,7 @@ public class ReplicationControlManagerTest { PartitionChangeRecord partitionChangeRecord = (PartitionChangeRecord) record.message(); assertEquals(0, partitionChangeRecord.partitionId()); assertEquals(2, partitionChangeRecord.leader()); - assertEquals(singletonList(2), partitionChangeRecord.isr()); + assertEquals(List.of(2), partitionChangeRecord.isr()); ctx.replay(result.records()); assertLeaderAndIsr(replication, partition0, 2, new int[]{2}); @@ -2558,14 +2554,14 @@ public class ReplicationControlManagerTest { ElectLeadersRequestData request = buildElectLeadersRequest( ElectionType.PREFERRED, - singletonMap("foo", singletonList(0)) + Map.of("foo", List.of(0)) ); // No election should be done even though unclean election is available ControllerResult result = replication.electLeaders(request); - assertEquals(Collections.emptyList(), result.records()); + assertEquals(List.of(), result.records()); - ElectLeadersResponseData expectedResponse = buildElectLeadersResponse(NONE, false, singletonMap( + ElectLeadersResponseData expectedResponse = buildElectLeadersResponse(NONE, false, Map.of( new TopicPartition("foo", 0), new ApiError(PREFERRED_LEADER_NOT_AVAILABLE) )); assertEquals(expectedResponse, result.response()); @@ -2632,11 +2628,11 @@ public class ReplicationControlManagerTest { new int[]{1, 2, 3}, new int[]{2, 3, 4}, new int[]{0, 2, 1}}).topicId(); ElectLeadersRequestData request1 = new ElectLeadersRequestData(). setElectionType(ElectionType.PREFERRED.value). - setTopicPartitions(new TopicPartitionsCollection(asList( + setTopicPartitions(new TopicPartitionsCollection(List.of( new TopicPartitions().setTopic("foo"). - setPartitions(asList(0, 1, 2)), + setPartitions(List.of(0, 1, 2)), new TopicPartitions().setTopic("bar"). - setPartitions(asList(0, 1))).iterator())); + setPartitions(List.of(0, 1))).iterator())); ControllerResult election1Result = replication.electLeaders(request1); ElectLeadersResponseData expectedResponse1 = buildElectLeadersResponse(NONE, false, Utils.mkMap( @@ -2662,7 +2658,7 @@ public class ReplicationControlManagerTest { ) )); assertElectLeadersResponse(expectedResponse1, election1Result.response()); - assertEquals(Collections.emptyList(), election1Result.records()); + assertEquals(List.of(), election1Result.records()); // Broker 1 must be registered to get out from the controlled shutdown state. ctx.registerBrokers(1); @@ -2671,28 +2667,28 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(singletonList(new TopicData().setTopicId(fooId). - setPartitions(asList( + setTopics(List.of(new TopicData().setTopicId(fooId). + setPartitions(List.of( new PartitionData(). setPartitionIndex(0).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsrWithEpochs(isrWithDefaultEpoch(1, 2, 3)), new PartitionData(). setPartitionIndex(2).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsrWithEpochs(isrWithDefaultEpoch(0, 2, 1))))))); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( - new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(asList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(2). setLeaderEpoch(0). - setIsr(asList(1, 2, 3)). + setIsr(List.of(1, 2, 3)). setPartitionEpoch(1). setErrorCode(NONE.code()), new AlterPartitionResponseData.PartitionData(). setPartitionIndex(2). setLeaderId(2). setLeaderEpoch(0). - setIsr(asList(0, 2, 1)). + setIsr(List.of(0, 2, 1)). setPartitionEpoch(1). setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -2725,7 +2721,7 @@ public class ReplicationControlManagerTest { replication.electLeaders(request1); assertElectLeadersResponse(expectedResponse2, election2Result.response()); assertEquals( - asList( + List.of( new ApiMessageAndVersion( new PartitionChangeRecord(). setPartitionId(0). @@ -2757,17 +2753,17 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(singletonList(new TopicData().setTopicId(fooId). - setPartitions(singletonList(new PartitionData(). + setTopics(List.of(new TopicData().setTopicId(fooId). + setPartitions(List.of(new PartitionData(). setPartitionIndex(0).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsrWithEpochs(isrWithDefaultEpoch(1, 2, 3))))))); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( - new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setLeaderId(2). setLeaderEpoch(0). - setIsr(asList(1, 2, 3)). + setIsr(List.of(1, 2, 3)). setPartitionEpoch(1). setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -2780,7 +2776,7 @@ public class ReplicationControlManagerTest { .setPartitionId(0) .setTopicId(fooId) .setLeader(1); - assertEquals(singletonList(new ApiMessageAndVersion(expectedChangeRecord, MetadataVersion.latestTesting().partitionChangeRecordVersion())), balanceResult.records()); + assertEquals(List.of(new ApiMessageAndVersion(expectedChangeRecord, MetadataVersion.latestTesting().partitionChangeRecordVersion())), balanceResult.records()); assertTrue(replication.arePartitionLeadersImbalanced()); assertFalse(balanceResult.response()); @@ -2789,17 +2785,17 @@ public class ReplicationControlManagerTest { alterPartitionResult = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequestData().setBrokerId(2).setBrokerEpoch(102). - setTopics(singletonList(new TopicData().setTopicId(fooId). - setPartitions(singletonList(new PartitionData(). + setTopics(List.of(new TopicData().setTopicId(fooId). + setPartitions(List.of(new PartitionData(). setPartitionIndex(2).setPartitionEpoch(0). setLeaderEpoch(0).setNewIsrWithEpochs(isrWithDefaultEpoch(0, 2, 1))))))); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( - new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( + new AlterPartitionResponseData.TopicData().setTopicId(fooId).setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(2). setLeaderId(2). setLeaderEpoch(0). - setIsr(asList(0, 2, 1)). + setIsr(List.of(0, 2, 1)). setPartitionEpoch(1). setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -2812,7 +2808,7 @@ public class ReplicationControlManagerTest { .setPartitionId(2) .setTopicId(fooId) .setLeader(0); - assertEquals(singletonList(new ApiMessageAndVersion(expectedChangeRecord, MetadataVersion.latestTesting().partitionChangeRecordVersion())), balanceResult.records()); + assertEquals(List.of(new ApiMessageAndVersion(expectedChangeRecord, MetadataVersion.latestTesting().partitionChangeRecordVersion())), balanceResult.records()); assertFalse(replication.arePartitionLeadersImbalanced()); assertFalse(balanceResult.response()); } @@ -2833,7 +2829,7 @@ public class ReplicationControlManagerTest { assertFalse(replication.areSomePartitionsLeaderless()); ctx.fenceBrokers(0, 1, 2, 3, 4); assertTrue(replication.areSomePartitionsLeaderless()); - for (int partitionId : Arrays.asList(0, 1, 2)) { + for (int partitionId : List.of(0, 1, 2)) { assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, partitionId).isr); assertEquals(-1, ctx.replicationControl.getPartition(fooId, partitionId).leader); } @@ -2852,7 +2848,7 @@ public class ReplicationControlManagerTest { assertEquals(2, ctx.replicationControl.getPartition(fooId, 2).leader); } else { // Otherwise, check that the election did NOT happen. - for (int partitionId : Arrays.asList(0, 1, 2)) { + for (int partitionId : List.of(0, 1, 2)) { assertArrayEquals(new int[] {4}, ctx.replicationControl.getPartition(fooId, partitionId).isr); assertEquals(-1, ctx.replicationControl.getPartition(fooId, partitionId).leader); } @@ -2861,20 +2857,20 @@ public class ReplicationControlManagerTest { // If we're setting unclean leader election dynamically, do that here. if (uncleanConfig.equals("dynamic_cluster")) { ctx.replay(ctx.configurationControl.incrementalAlterConfigs( - Collections.singletonMap(new ConfigResource(ConfigResource.Type.BROKER, ""), - Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + Map.of(new ConfigResource(ConfigResource.Type.BROKER, ""), + Map.of(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), true).records()); } else if (uncleanConfig.equals("dynamic_node")) { ctx.replay(ctx.configurationControl.incrementalAlterConfigs( - Collections.singletonMap(new ConfigResource(ConfigResource.Type.BROKER, "0"), - Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + Map.of(new ConfigResource(ConfigResource.Type.BROKER, "0"), + Map.of(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), true).records()); } else if (uncleanConfig.equals("dynamic_topic")) { ctx.replay(ctx.configurationControl.incrementalAlterConfigs( - Collections.singletonMap(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), - Collections.singletonMap(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, + Map.of(new ConfigResource(ConfigResource.Type.TOPIC, "foo"), + Map.of(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "true"))), true).records()); } @@ -2951,11 +2947,11 @@ public class ReplicationControlManagerTest { ReplicationControlTestContext ctx = new ReplicationControlTestContext.Builder().build(); ReplicationControlManager replication = ctx.replicationControl; ctx.registerBrokersWithDirs( - 0, Collections.emptyList(), - 1, Collections.emptyList(), - 2, asList(Uuid.fromString("ozwqsVMFSNiYQUPSJA3j0w")), - 3, asList(Uuid.fromString("SSDgCZ4BTyec5QojGT65qg"), Uuid.fromString("K8KwMrviRcOUvgI8FPOJWg")), - 4, Collections.emptyList() + 0, List.of(), + 1, List.of(), + 2, List.of(Uuid.fromString("ozwqsVMFSNiYQUPSJA3j0w")), + 3, List.of(Uuid.fromString("SSDgCZ4BTyec5QojGT65qg"), Uuid.fromString("K8KwMrviRcOUvgI8FPOJWg")), + 4, List.of() ); ctx.unfenceBrokers(2, 3, 4); ctx.createTestTopic("foo", new int[][]{ @@ -2965,7 +2961,7 @@ public class ReplicationControlManagerTest { KRaftClusterDescriber describer = replication.clusterDescriber; HashSet brokers = new HashSet<>(); describer.usableBrokers().forEachRemaining(broker -> brokers.add(broker)); - assertEquals(new HashSet<>(Arrays.asList( + assertEquals(new HashSet<>(List.of( new UsableBroker(0, Optional.empty(), true), new UsableBroker(1, Optional.empty(), true), new UsableBroker(2, Optional.empty(), false), @@ -3009,7 +3005,7 @@ public class ReplicationControlManagerTest { new PartitionChangeRecord() .setPartitionId(0) .setTopicId(topicId) - .setIsr(asList(1, 2)) + .setIsr(List.of(1, 2)) .setLeader(1), (short) 0)); @@ -3058,25 +3054,25 @@ public class ReplicationControlManagerTest { // Reassign to [2, 3] ControllerResult alterResultOne = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName(topic).setPartitions(singletonList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName(topic).setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(2, 3))))))); + setReplicas(List.of(2, 3))))))); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setResponses(singletonList( - new ReassignableTopicResponse().setName(topic).setPartitions(singletonList( + setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName(topic).setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null))))), alterResultOne.response()); ctx.replay(alterResultOne.records()); ListPartitionReassignmentsResponseData currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName(topic).setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName(topic).setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(0). - setRemovingReplicas(asList(0, 1)). - setAddingReplicas(asList(2, 3)). - setReplicas(asList(2, 3, 0, 1)))))); + setRemovingReplicas(List.of(0, 1)). + setAddingReplicas(List.of(2, 3)). + setReplicas(List.of(2, 3, 0, 1)))))); // Make sure the reassignment metadata is as expected. assertEquals(currentReassigning, replication.listPartitionReassignments(null, Long.MAX_VALUE)); @@ -3087,9 +3083,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterPartitionRequestData = new AlterPartitionRequestData(). setBrokerId(partition.leader). setBrokerEpoch(ctx.currentBrokerEpoch(partition.leader)). - setTopics(singletonList(new TopicData(). + setTopics(List.of(new TopicData(). setTopicId(topicId). - setPartitions(singletonList(new PartitionData(). + setPartitions(List.of(new PartitionData(). setPartitionIndex(0). setPartitionEpoch(partition.partitionEpoch). setLeaderEpoch(partition.leaderEpoch). @@ -3097,13 +3093,13 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResult = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequest.Builder(alterPartitionRequestData).build().data()); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( new AlterPartitionResponseData.TopicData(). setTopicId(topicId). - setPartitions(singletonList( + setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). - setIsr(asList(0, 1, 2)). + setIsr(List.of(0, 1, 2)). setPartitionEpoch(partition.partitionEpoch + 1). setErrorCode(NONE.code()))))), alterPartitionResult.response()); @@ -3113,13 +3109,13 @@ public class ReplicationControlManagerTest { // Elect replica 2 as leader via preferred leader election. 2 is at the front of the replicas list. ElectLeadersRequestData request = buildElectLeadersRequest( ElectionType.PREFERRED, - singletonMap(topic, singletonList(0)) + Map.of(topic, List.of(0)) ); ControllerResult electLeaderTwoResult = replication.electLeaders(request); ReplicaElectionResult replicaElectionResult = new ReplicaElectionResult().setTopic(topic); - replicaElectionResult.setPartitionResult(singletonList(new PartitionResult().setPartitionId(0).setErrorCode(NONE.code()).setErrorMessage(null))); + replicaElectionResult.setPartitionResult(List.of(new PartitionResult().setPartitionId(0).setErrorCode(NONE.code()).setErrorMessage(null))); assertEquals( - new ElectLeadersResponseData().setErrorCode(NONE.code()).setReplicaElectionResults(singletonList(replicaElectionResult)), + new ElectLeadersResponseData().setErrorCode(NONE.code()).setReplicaElectionResults(List.of(replicaElectionResult)), electLeaderTwoResult.response() ); ctx.replay(electLeaderTwoResult.records()); @@ -3130,13 +3126,13 @@ public class ReplicationControlManagerTest { // Reassign to [4, 5] ControllerResult alterResultTwo = replication.alterPartitionReassignments( - new AlterPartitionReassignmentsRequestData().setTopics(singletonList( - new ReassignableTopic().setName(topic).setPartitions(singletonList( + new AlterPartitionReassignmentsRequestData().setTopics(List.of( + new ReassignableTopic().setName(topic).setPartitions(List.of( new ReassignablePartition().setPartitionIndex(0). - setReplicas(asList(4, 5))))))); + setReplicas(List.of(4, 5))))))); assertEquals(new AlterPartitionReassignmentsResponseData(). - setErrorMessage(null).setResponses(singletonList( - new ReassignableTopicResponse().setName(topic).setPartitions(singletonList( + setErrorMessage(null).setResponses(List.of( + new ReassignableTopicResponse().setName(topic).setPartitions(List.of( new ReassignablePartitionResponse().setPartitionIndex(0). setErrorMessage(null))))), alterResultTwo.response()); ctx.replay(alterResultTwo.records()); @@ -3144,12 +3140,12 @@ public class ReplicationControlManagerTest { // Make sure the replicas list contains all the previous replicas 0, 1, 2, 3 as well as the new replicas 3, 4 currentReassigning = new ListPartitionReassignmentsResponseData().setErrorMessage(null). - setTopics(singletonList(new OngoingTopicReassignment(). - setName(topic).setPartitions(singletonList( + setTopics(List.of(new OngoingTopicReassignment(). + setName(topic).setPartitions(List.of( new OngoingPartitionReassignment().setPartitionIndex(0). - setRemovingReplicas(asList(0, 1, 2, 3)). - setAddingReplicas(asList(4, 5)). - setReplicas(asList(4, 5, 0, 1, 2, 3)))))); + setRemovingReplicas(List.of(0, 1, 2, 3)). + setAddingReplicas(List.of(4, 5)). + setReplicas(List.of(4, 5, 0, 1, 2, 3)))))); assertEquals(currentReassigning, replication.listPartitionReassignments(null, Long.MAX_VALUE)); @@ -3162,9 +3158,9 @@ public class ReplicationControlManagerTest { AlterPartitionRequestData alterPartitionRequestDataTwo = new AlterPartitionRequestData(). setBrokerId(partition.leader). setBrokerEpoch(ctx.currentBrokerEpoch(partition.leader)). - setTopics(singletonList(new TopicData(). + setTopics(List.of(new TopicData(). setTopicId(topicId). - setPartitions(singletonList(new PartitionData(). + setPartitions(List.of(new PartitionData(). setPartitionIndex(0). setPartitionEpoch(partition.partitionEpoch). setLeaderEpoch(partition.leaderEpoch). @@ -3172,10 +3168,10 @@ public class ReplicationControlManagerTest { ControllerResult alterPartitionResultTwo = replication.alterPartition( anonymousContextFor(ApiKeys.ALTER_PARTITION), new AlterPartitionRequest.Builder(alterPartitionRequestDataTwo).build().data()); - assertEquals(new AlterPartitionResponseData().setTopics(singletonList( + assertEquals(new AlterPartitionResponseData().setTopics(List.of( new AlterPartitionResponseData.TopicData(). setTopicId(topicId). - setPartitions(singletonList( + setPartitions(List.of( new AlterPartitionResponseData.PartitionData(). setPartitionIndex(0). setErrorCode(NEW_LEADER_ELECTED.code()))))), @@ -3239,7 +3235,7 @@ public class ReplicationControlManagerTest { Uuid dir2b1 = Uuid.fromString("R3Gb1HLoTzuKMgAkH5Vtpw"); Uuid dir1b2 = Uuid.fromString("TBGa8UayQi6KguqF5nC0sw"); Uuid offlineDir = Uuid.fromString("zvAf9BKZRyyrEWz4FX2nLA"); - ctx.registerBrokersWithDirs(1, asList(dir1b1, dir2b1), 2, singletonList(dir1b2)); + ctx.registerBrokersWithDirs(1, List.of(dir1b1, dir2b1), 2, List.of(dir1b2)); ctx.unfenceBrokers(1, 2); Uuid topicA = ctx.createTestTopic("a", new int[][]{new int[]{1, 2}, new int[]{1, 2}, new int[]{1, 2}}).topicId(); Uuid topicB = ctx.createTestTopic("b", new int[][]{new int[]{1, 2}, new int[]{1, 2}}).topicId(); @@ -3275,22 +3271,22 @@ public class ReplicationControlManagerTest { }}); }})), AssignmentsHelper.normalize(controllerResult.response())); short recordVersion = ctx.featureControl.metadataVersionOrThrow().partitionChangeRecordVersion(); - assertEquals(sortPartitionChangeRecords(asList( + assertEquals(sortPartitionChangeRecords(List.of( new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicA).setPartitionId(0) - .setDirectories(asList(dir1b1, dir1b2)), recordVersion), + .setDirectories(List.of(dir1b1, dir1b2)), recordVersion), new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicA).setPartitionId(1). - setDirectories(asList(dir2b1, dir1b2)), recordVersion), + setDirectories(List.of(dir2b1, dir1b2)), recordVersion), new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicA).setPartitionId(2). - setDirectories(asList(offlineDir, dir1b2)), recordVersion), + setDirectories(List.of(offlineDir, dir1b2)), recordVersion), new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicB).setPartitionId(0). - setDirectories(asList(dir1b1, dir1b2)), recordVersion), + setDirectories(List.of(dir1b1, dir1b2)), recordVersion), new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicB).setPartitionId(1). - setDirectories(asList(DirectoryId.LOST, dir1b2)), recordVersion), + setDirectories(List.of(DirectoryId.LOST, dir1b2)), recordVersion), // In addition to the directory assignment changes we expect two additional records, // which elect new leaders for: @@ -3298,10 +3294,10 @@ public class ReplicationControlManagerTest { // - b-1 which has been assigned to an offline directory. new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicA).setPartitionId(2). - setIsr(singletonList(2)).setLeader(2), recordVersion), + setIsr(List.of(2)).setLeader(2), recordVersion), new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(topicB).setPartitionId(1). - setIsr(singletonList(2)).setLeader(2), recordVersion) + setIsr(List.of(2)).setLeader(2), recordVersion) )), sortPartitionChangeRecords(controllerResult.records())); ctx.replay(controllerResult.records()); @@ -3326,7 +3322,7 @@ public class ReplicationControlManagerTest { Uuid dir2b1 = Uuid.fromString("yh3acnzGSeurSTj8aIhOjw"); Uuid dir1b2 = Uuid.fromString("OmpmJ8RjQliQlEFht56DwQ"); Uuid dir2b2 = Uuid.fromString("w05baLpsT5Oz0LvKTKXoDw"); - ctx.registerBrokersWithDirs(b1, asList(dir1b1, dir2b1), b2, asList(dir1b2, dir2b2)); + ctx.registerBrokersWithDirs(b1, List.of(dir1b1, dir2b1), b2, List.of(dir1b2, dir2b2)); ctx.unfenceBrokers(b1, b2); Uuid topicA = ctx.createTestTopic("a", new int[][]{new int[]{b1, b2}, new int[]{b1, b2}}).topicId(); Uuid topicB = ctx.createTestTopic("b", new int[][]{new int[]{b1, b2}, new int[]{b1, b2}}).topicId(); @@ -3343,29 +3339,29 @@ public class ReplicationControlManagerTest { put(new TopicIdPartition(topicB, 1), dir2b2); }}); List records = new ArrayList<>(); - ctx.replicationControl.handleDirectoriesOffline(b1, defaultBrokerEpoch(b1), asList( + ctx.replicationControl.handleDirectoriesOffline(b1, defaultBrokerEpoch(b1), List.of( dir1b1, dir1b2 // should not cause update to dir1b2 as it's not registered to b1 ), records); assertEquals( - singletonList(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord() + List.of(new ApiMessageAndVersion(new BrokerRegistrationChangeRecord() .setBrokerId(b1).setBrokerEpoch(defaultBrokerEpoch(b1)) - .setLogDirs(singletonList(dir2b1)), (short) 2)), + .setLogDirs(List.of(dir2b1)), (short) 2)), filter(records, BrokerRegistrationChangeRecord.class) ); short partitionChangeRecordVersion = ctx.featureControl.metadataVersionOrThrow().partitionChangeRecordVersion(); assertEquals( - sortPartitionChangeRecords(asList( + sortPartitionChangeRecords(List.of( new ApiMessageAndVersion(new PartitionChangeRecord().setTopicId(topicA).setPartitionId(0) - .setLeader(b2).setIsr(singletonList(b2)), partitionChangeRecordVersion), + .setLeader(b2).setIsr(List.of(b2)), partitionChangeRecordVersion), new ApiMessageAndVersion(new PartitionChangeRecord().setTopicId(topicB).setPartitionId(0) - .setLeader(b2).setIsr(singletonList(b2)), partitionChangeRecordVersion) + .setLeader(b2).setIsr(List.of(b2)), partitionChangeRecordVersion) )), sortPartitionChangeRecords(filter(records, PartitionChangeRecord.class)) ); assertEquals(3, records.size()); ctx.replay(records); - assertEquals(Collections.singletonList(dir2b1), ctx.clusterControl.registration(b1).directories()); + assertEquals(List.of(dir2b1), ctx.clusterControl.registration(b1).directories()); } /** @@ -3410,13 +3406,13 @@ public class ReplicationControlManagerTest { } if (useLegacyAlterConfigs) { ctx.replay(ctx.configurationControl.legacyAlterConfigs( - Collections.singletonMap(configResource, - Collections.singletonMap(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")), + Map.of(configResource, + Map.of(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "1")), false).records()); } else { ctx.replay(ctx.configurationControl.incrementalAlterConfigs( - Collections.singletonMap(configResource, - Collections.singletonMap(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, + Map.of(configResource, + Map.of(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, new AbstractMap.SimpleImmutableEntry<>(AlterConfigOp.OpType.SET, "1"))), false).records()); } @@ -3439,7 +3435,7 @@ public class ReplicationControlManagerTest { Uuid fooId = ctx.createTestTopic("foo", new int[][]{ new int[]{1, 2, 4}, new int[]{1, 3, 4}}).topicId(); int partitionEpoch = ctx.replicationControl.getPartition(fooId, 0).partitionEpoch; - ctx.replay(Arrays.asList(new ApiMessageAndVersion(new ClearElrRecord(), CLEAR_ELR_RECORD.highestSupportedVersion()))); + ctx.replay(List.of(new ApiMessageAndVersion(new ClearElrRecord(), CLEAR_ELR_RECORD.highestSupportedVersion()))); assertEquals(partitionEpoch, ctx.replicationControl.getPartition(fooId, 0).partitionEpoch); } } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java index 47671c2504f..42801c510f1 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetadataMetricsTest.java @@ -24,10 +24,10 @@ import com.yammer.metrics.core.MetricsRegistry; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.function.BiConsumer; import java.util.function.Function; @@ -40,7 +40,7 @@ public class ControllerMetadataMetricsTest { try { try (ControllerMetadataMetrics metrics = new ControllerMetadataMetrics(Optional.of(registry))) { ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller:", - new HashSet<>(Arrays.asList( + new HashSet<>(List.of( "kafka.controller:type=KafkaController,name=ActiveBrokerCount", "kafka.controller:type=KafkaController,name=FencedBrokerCount", "kafka.controller:type=KafkaController,name=GlobalPartitionCount", @@ -53,7 +53,7 @@ public class ControllerMetadataMetricsTest { ))); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "KafkaController", - Collections.emptySet()); + Set.of()); } finally { registry.shutdown(); } diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java index c172725c67a..1964c93c8b3 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/ControllerMetricsChangesTest.java @@ -29,7 +29,6 @@ import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -131,7 +130,7 @@ public class ControllerMetricsChangesTest { static { ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_3_7_IV0).build(); // highest MV for PartitionRecord v0 - TOPIC_DELTA1 = new TopicDelta(new TopicImage("foo", FOO_ID, Collections.emptyMap())); + TOPIC_DELTA1 = new TopicDelta(new TopicImage("foo", FOO_ID, Map.of())); TOPIC_DELTA1.replay((PartitionRecord) fakePartitionRegistration(NORMAL). toRecord(FOO_ID, 0, options).message()); TOPIC_DELTA1.replay((PartitionRecord) fakePartitionRegistration(NORMAL). diff --git a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java index 041f3c8312b..8e1ee11f7f9 100644 --- a/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/controller/metrics/QuorumControllerMetricsTest.java @@ -26,10 +26,10 @@ import com.yammer.metrics.core.MetricsRegistry; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -42,7 +42,7 @@ public class QuorumControllerMetricsTest { try (QuorumControllerMetrics metrics = new QuorumControllerMetrics( Optional.of(registry), time)) { - HashSet expected = new HashSet<>(Arrays.asList( + HashSet expected = new HashSet<>(List.of( "kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs", "kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs", "kafka.controller:type=KafkaController,name=ActiveControllerCount", @@ -58,7 +58,7 @@ public class QuorumControllerMetricsTest { ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", expected); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", - Collections.emptySet()); + Set.of()); } finally { registry.shutdown(); } diff --git a/metadata/src/test/java/org/apache/kafka/image/AclsDeltaTest.java b/metadata/src/test/java/org/apache/kafka/image/AclsDeltaTest.java index 1d403b5cc42..43bfedba359 100644 --- a/metadata/src/test/java/org/apache/kafka/image/AclsDeltaTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/AclsDeltaTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.metadata.authorizer.StandardAcl; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -42,7 +41,7 @@ public class AclsDeltaTest { @Test public void testRemovesDeleteIfNotInImage() { - AclsImage image = new AclsImage(Collections.emptyMap()); + AclsImage image = new AclsImage(Map.of()); AclsDelta delta = new AclsDelta(image); AccessControlEntryRecord inputAclRecord = testAccessControlEntryRecord(); @@ -75,7 +74,7 @@ public class AclsDeltaTest { @Test public void testThrowsExceptionOnInvalidStateWhenImageIsEmpty() { - AclsImage image = new AclsImage(Collections.emptyMap()); + AclsImage image = new AclsImage(Map.of()); AclsDelta delta = new AclsDelta(image); RemoveAccessControlEntryRecord removeAccessControlEntryRecord = testRemoveAccessControlEntryRecord(); diff --git a/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java index d1d33444989..76a878e1d48 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ClientQuotasImageTest.java @@ -31,8 +31,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -54,33 +52,33 @@ public class ClientQuotasImageTest { static { Map entities1 = new HashMap<>(); - Map fooUser = Collections.singletonMap(ClientQuotaEntity.USER, "foo"); - Map fooUserQuotas = Collections.singletonMap(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 123.0); + Map fooUser = Map.of(ClientQuotaEntity.USER, "foo"); + Map fooUserQuotas = Map.of(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 123.0); entities1.put(new ClientQuotaEntity(fooUser), new ClientQuotaImage(fooUserQuotas)); Map barUserAndIp = new HashMap<>(); barUserAndIp.put(ClientQuotaEntity.USER, "bar"); barUserAndIp.put(ClientQuotaEntity.IP, "127.0.0.1"); - Map barUserAndIpQuotas = Collections.singletonMap(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 456.0); + Map barUserAndIpQuotas = Map.of(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 456.0); entities1.put(new ClientQuotaEntity(barUserAndIp), new ClientQuotaImage(barUserAndIpQuotas)); IMAGE1 = new ClientQuotasImage(entities1); DELTA1_RECORDS = new ArrayList<>(); // remove quota DELTA1_RECORDS.add(new ApiMessageAndVersion(new ClientQuotaRecord(). - setEntity(Arrays.asList( + setEntity(List.of( new EntityData().setEntityType(ClientQuotaEntity.USER).setEntityName("bar"), new EntityData().setEntityType(ClientQuotaEntity.IP).setEntityName("127.0.0.1"))). setKey(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG). setRemove(true), CLIENT_QUOTA_RECORD.highestSupportedVersion())); // alter quota DELTA1_RECORDS.add(new ApiMessageAndVersion(new ClientQuotaRecord(). - setEntity(Collections.singletonList( + setEntity(List.of( new EntityData().setEntityType(ClientQuotaEntity.USER).setEntityName("foo"))). setKey(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG). setValue(234.0), CLIENT_QUOTA_RECORD.highestSupportedVersion())); // add quota to entity with existing quota DELTA1_RECORDS.add(new ApiMessageAndVersion(new ClientQuotaRecord(). - setEntity(Collections.singletonList( + setEntity(List.of( new EntityData().setEntityType(ClientQuotaEntity.USER).setEntityName("foo"))). setKey(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG). setValue(999.0), CLIENT_QUOTA_RECORD.highestSupportedVersion())); diff --git a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java index 7e0cb9905fc..4d26e857baa 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ClusterImageTest.java @@ -46,11 +46,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; 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.atomic.AtomicReference; import static org.apache.kafka.common.metadata.MetadataRecordType.BROKER_REGISTRATION_CHANGE_RECORD; @@ -83,8 +83,8 @@ public class ClusterImageTest { setId(0). setEpoch(1000). setIncarnationId(Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(true). setInControlledShutdown(false).build()); @@ -92,8 +92,8 @@ public class ClusterImageTest { setId(1). setEpoch(1001). setIncarnationId(Uuid.fromString("U52uRe20RsGI0RvpcTx33Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(false).build()); @@ -101,8 +101,8 @@ public class ClusterImageTest { setId(2). setEpoch(123). setIncarnationId(Uuid.fromString("hr4TVh3YQiu3p16Awkka6w")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). - setSupportedFeatures(Collections.emptyMap()). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). + setSupportedFeatures(Map.of()). setRack(Optional.of("arack")). setFenced(false). setInControlledShutdown(false).build()); @@ -111,9 +111,9 @@ public class ClusterImageTest { setId(1000). setIncarnationId(Uuid.fromString("9ABu6HEgRuS-hjHLgC4cHw")). setZkMigrationReady(false). - setListeners(Collections.singletonMap("PLAINTEXT", + setListeners(Map.of("PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 19092))). - setSupportedFeatures(Collections.emptyMap()).build()); + setSupportedFeatures(Map.of()).build()); IMAGE1 = new ClusterImage(map1, cmap1); DELTA1_RECORDS = new ArrayList<>(); @@ -154,8 +154,8 @@ public class ClusterImageTest { setId(0). setEpoch(1000). setIncarnationId(Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(true).build()); @@ -163,8 +163,8 @@ public class ClusterImageTest { setId(1). setEpoch(1001). setIncarnationId(Uuid.fromString("U52uRe20RsGI0RvpcTx33Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(true). setInControlledShutdown(false).build()); @@ -173,9 +173,9 @@ public class ClusterImageTest { setId(1001). setIncarnationId(Uuid.fromString("FdEHF-IqScKfYyjZ1CjfNQ")). setZkMigrationReady(true). - setListeners(Collections.singletonMap("PLAINTEXT", + setListeners(Map.of("PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 19093))). - setSupportedFeatures(Collections.emptyMap()).build()); + setSupportedFeatures(Map.of()).build()); IMAGE2 = new ClusterImage(map2, cmap2); DELTA2_RECORDS = new ArrayList<>(DELTA1_RECORDS); @@ -194,10 +194,10 @@ public class ClusterImageTest { DELTA2_RECORDS.add(new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(2).setIsMigratingZkBroker(true).setIncarnationId(Uuid.fromString("Am5Yse7GQxaw0b2alM74bP")). setBrokerEpoch(1002).setEndPoints(new BrokerEndpointCollection( - Collections.singletonList(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). + List.of(new BrokerEndpoint().setName("PLAINTEXT").setHost("localhost"). setPort(9094).setSecurityProtocol((short) 0)).iterator())). setFeatures(new BrokerFeatureCollection( - Collections.singleton(new BrokerFeature(). + Set.of(new BrokerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). setMaxSupportedVersion(MetadataVersion.IBP_3_6_IV0.featureLevel())).iterator())). @@ -212,8 +212,8 @@ public class ClusterImageTest { setId(0). setEpoch(1000). setIncarnationId(Uuid.fromString("vZKYST0pSA2HO5x_6hoO2Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9092))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(true). setInControlledShutdown(true).build()); @@ -221,8 +221,8 @@ public class ClusterImageTest { setId(1). setEpoch(1001). setIncarnationId(Uuid.fromString("U52uRe20RsGI0RvpcTx33Q")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 3))). + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9093))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 3))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(false).build()); @@ -230,8 +230,8 @@ public class ClusterImageTest { setId(2). setEpoch(1002). setIncarnationId(Uuid.fromString("Am5Yse7GQxaw0b2alM74bP")). - setListeners(Collections.singletonList(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). - setSupportedFeatures(Collections.singletonMap("metadata.version", + setListeners(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9094))). + setSupportedFeatures(Map.of("metadata.version", VersionRange.of(MetadataVersion.MINIMUM_VERSION.featureLevel(), MetadataVersion.IBP_3_6_IV0.featureLevel()))). setRack(Optional.of("rack3")). setFenced(true). @@ -302,13 +302,13 @@ public class ClusterImageTest { @Test public void testHandleLossOfControllerRegistrations() { - ClusterImage testImage = new ClusterImage(Collections.emptyMap(), - Collections.singletonMap(1000, new ControllerRegistration.Builder(). + ClusterImage testImage = new ClusterImage(Map.of(), + Map.of(1000, new ControllerRegistration.Builder(). setId(1000). setIncarnationId(Uuid.fromString("9ABu6HEgRuS-hjHLgC4cHw")). - setListeners(Collections.singletonMap("PLAINTEXT", + setListeners(Map.of("PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 19092))). - setSupportedFeatures(Collections.emptyMap()).build())); + setSupportedFeatures(Map.of()).build())); RecordListWriter writer = new RecordListWriter(); final AtomicReference lossString = new AtomicReference<>(""); testImage.write(writer, new ImageWriterOptions.Builder(MetadataVersion.IBP_3_6_IV2). diff --git a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java index 2135a498bf4..6e01c4dbf03 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ConfigurationsImageTest.java @@ -29,7 +29,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -87,10 +86,10 @@ public class ConfigurationsImageTest { RecordTestUtils.replayAll(DELTA1, DELTA1_RECORDS); Map map2 = new HashMap<>(); - Map broker1Map2 = Collections.singletonMap("barfoo", "bazfoo"); + Map broker1Map2 = Map.of("barfoo", "bazfoo"); map2.put(new ConfigResource(BROKER, "1"), new ConfigurationImage(new ConfigResource(BROKER, "1"), broker1Map2)); - Map broker2Map = Collections.singletonMap("foo", "bar"); + Map broker2Map = Map.of("foo", "bar"); map2.put(new ConfigResource(BROKER, "2"), new ConfigurationImage(new ConfigResource(BROKER, "2"), broker2Map)); IMAGE2 = new ConfigurationsImage(map2); } diff --git a/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java b/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java index f3dabe8d793..43c999c04ab 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java +++ b/metadata/src/test/java/org/apache/kafka/image/FakeSnapshotWriter.java @@ -22,7 +22,6 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.snapshot.SnapshotWriter; import java.util.ArrayList; -import java.util.Collections; import java.util.List; @@ -33,11 +32,7 @@ public class FakeSnapshotWriter implements SnapshotWriter private boolean closed = false; public List> batches() { - List> result = new ArrayList<>(); - for (List batch : batches) { - result.add(Collections.unmodifiableList(batch)); - } - return Collections.unmodifiableList(result); + return batches.stream().map(List::copyOf).toList(); } public FakeSnapshotWriter() { diff --git a/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java b/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java index ff5deba81ca..0e42e2ac8f8 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/FeaturesDeltaTest.java @@ -24,7 +24,8 @@ import org.apache.kafka.server.common.MetadataVersionTestUtils; import org.junit.jupiter.api.Test; -import static java.util.Collections.emptyMap; +import java.util.Map; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -33,7 +34,7 @@ class FeaturesDeltaTest { @Test public void testReplayWithUnsupportedFeatureLevel() { - var featuresDelta = new FeaturesDelta(new FeaturesImage(emptyMap(), MetadataVersion.MINIMUM_VERSION)); + var featuresDelta = new FeaturesDelta(new FeaturesImage(Map.of(), MetadataVersion.MINIMUM_VERSION)); var exception = assertThrows(IllegalArgumentException.class, () -> featuresDelta.replay(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) .setFeatureLevel(MetadataVersionTestUtils.IBP_3_3_IV2_FEATURE_LEVEL))); @@ -44,8 +45,8 @@ class FeaturesDeltaTest { @Test public void testReplayKraftVersionFeatureLevel() { - var featuresDelta = new FeaturesDelta(new FeaturesImage(emptyMap(), MetadataVersion.MINIMUM_VERSION)); + var featuresDelta = new FeaturesDelta(new FeaturesImage(Map.of(), MetadataVersion.MINIMUM_VERSION)); featuresDelta.replay(new FeatureLevelRecord().setName(KRaftVersion.FEATURE_NAME).setFeatureLevel(KRaftVersion.LATEST_PRODUCTION.featureLevel())); - assertEquals(emptyMap(), featuresDelta.changes()); + assertEquals(Map.of(), featuresDelta.changes()); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java index 39e11ae03ea..cbd54c769e4 100644 --- a/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/FeaturesImageTest.java @@ -29,7 +29,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -94,7 +93,7 @@ public class FeaturesImageTest { DELTA2 = new FeaturesDelta(IMAGE2); RecordTestUtils.replayAll(DELTA2, DELTA2_RECORDS); - Map map3 = Collections.singletonMap("bar", (short) 1); + Map map3 = Map.of("bar", (short) 1); IMAGE3 = new FeaturesImage(map3, MetadataVersion.latestTesting()); } @@ -105,7 +104,7 @@ public class FeaturesImageTest { RecordListWriter writer = new RecordListWriter(); image.write(writer, new ImageWriterOptions.Builder(metadataVersion).build()); // A metadata version is required for writing, so the expected image is not actually empty - var expectedImage = new FeaturesImage(Collections.emptyMap(), metadataVersion); + var expectedImage = new FeaturesImage(Map.of(), metadataVersion); testToImage(expectedImage, writer.records()); } @@ -167,7 +166,7 @@ public class FeaturesImageTest { @Test public void testEmpty() { assertTrue(FeaturesImage.EMPTY.isEmpty()); - assertFalse(new FeaturesImage(Collections.singletonMap("foo", (short) 1), + assertFalse(new FeaturesImage(Map.of("foo", (short) 1), MetadataVersion.MINIMUM_VERSION).isEmpty()); assertFalse(new FeaturesImage(FeaturesImage.EMPTY.finalizedVersions(), MetadataVersion.MINIMUM_VERSION).isEmpty()); diff --git a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java index 37bfa3462da..4a64fd8c19d 100644 --- a/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/ImageDowngradeTest.java @@ -33,8 +33,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.function.Consumer; @@ -59,14 +57,14 @@ public class ImageDowngradeTest { } } - static final List TEST_RECORDS = Arrays.asList( + static final List TEST_RECORDS = List.of( new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("5JPuABiJTPu2pQjpZWM6_A")), (short) 0), new ApiMessageAndVersion(new PartitionRecord(). setTopicId(Uuid.fromString("5JPuABiJTPu2pQjpZWM6_A")). - setReplicas(Arrays.asList(0, 1)). - setIsr(Arrays.asList(0, 1)). + setReplicas(List.of(0, 1)). + setIsr(List.of(0, 1)). setLeader(0). setLeaderEpoch(1). setPartitionEpoch(2), (short) 0)); @@ -83,9 +81,9 @@ public class ImageDowngradeTest { @Test public void testPreZkMigrationSupportVersion() { writeWithExpectedLosses(MetadataVersion.IBP_3_3_IV3, - Collections.singletonList( + List.of( "the isMigratingZkBroker state of one or more brokers"), - Arrays.asList( + List.of( metadataVersionRecord(MetadataVersion.IBP_3_4_IV0), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(123). @@ -97,7 +95,7 @@ public class ImageDowngradeTest { setIsMigratingZkBroker(true), (short) 2), TEST_RECORDS.get(0), TEST_RECORDS.get(1)), - Arrays.asList( + List.of( metadataVersionRecord(MetadataVersion.IBP_3_3_IV3), new ApiMessageAndVersion(new RegisterBrokerRecord(). setBrokerId(123). @@ -117,21 +115,21 @@ public class ImageDowngradeTest { MetadataVersion inputMetadataVersion = outputMetadataVersion; PartitionRecord testPartitionRecord = (PartitionRecord) TEST_RECORDS.get(1).message(); writeWithExpectedLosses(outputMetadataVersion, - Collections.singletonList( + List.of( "the directory assignment state of one or more replicas"), - Arrays.asList( + List.of( metadataVersionRecord(inputMetadataVersion), TEST_RECORDS.get(0), new ApiMessageAndVersion( - testPartitionRecord.duplicate().setDirectories(Arrays.asList( + testPartitionRecord.duplicate().setDirectories(List.of( Uuid.fromString("c7QfSi6xSIGQVh3Qd5RJxA"), Uuid.fromString("rWaCHejCRRiptDMvW5Xw0g"))), (short) 2)), - Arrays.asList( + List.of( metadataVersionRecord(outputMetadataVersion), TEST_RECORDS.get(0), new ApiMessageAndVersion( - testPartitionRecord.duplicate().setDirectories(Collections.emptyList()), + testPartitionRecord.duplicate().setDirectories(List.of()), (short) 0)) ); } diff --git a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java index 715ac0beed4..d4a091da809 100644 --- a/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/MetadataImageTest.java @@ -26,8 +26,8 @@ import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -88,7 +88,7 @@ public class MetadataImageTest { // A metadata version is required for writing, so the expected image is not actually empty var expectedImage = new MetadataImage( MetadataProvenance.EMPTY, - new FeaturesImage(Collections.emptyMap(), metadataVersion), + new FeaturesImage(Map.of(), metadataVersion), ClusterImage.EMPTY, TopicsImage.EMPTY, ConfigurationsImage.EMPTY, diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java index c868ebdfee7..b91612d70ba 100644 --- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java @@ -38,14 +38,13 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import static org.apache.kafka.common.metadata.MetadataRecordType.CLEAR_ELR_RECORD; import static org.apache.kafka.common.metadata.MetadataRecordType.PARTITION_CHANGE_RECORD; @@ -109,7 +108,7 @@ public class TopicsImageTest { private static final Uuid BAM_UUID2 = Uuid.fromString("yd6Sq3a9aK1G8snlKv7ag5"); static { - TOPIC_IMAGES1 = Arrays.asList( + TOPIC_IMAGES1 = List.of( newTopicImage("foo", FOO_UUID, new PartitionRegistration.Builder().setReplicas(new int[] {2, 3, 4}). setDirectories(DirectoryId.migratingArray(3)). @@ -145,10 +144,10 @@ public class TopicsImageTest { DELTA1_RECORDS.add(new ApiMessageAndVersion(new PartitionRecord(). setPartitionId(0). setTopicId(BAZ_UUID). - setReplicas(Arrays.asList(1, 2, 3, 4)). - setIsr(Arrays.asList(3, 4)). - setRemovingReplicas(Collections.singletonList(2)). - setAddingReplicas(Collections.singletonList(1)). + setReplicas(List.of(1, 2, 3, 4)). + setIsr(List.of(3, 4)). + setRemovingReplicas(List.of(2)). + setAddingReplicas(List.of(1)). setLeader(3). setLeaderEpoch(2). setPartitionEpoch(1), PARTITION_RECORD.highestSupportedVersion())); @@ -167,7 +166,7 @@ public class TopicsImageTest { DELTA1 = new TopicsDelta(IMAGE1); RecordTestUtils.replayAll(DELTA1, DELTA1_RECORDS); - List topics2 = Arrays.asList( + List topics2 = List.of( newTopicImage("foo", FOO_UUID2), newTopicImage("bar", BAR_UUID, new PartitionRegistration.Builder().setReplicas(new int[] {0, 1, 2, 3, 4}). @@ -228,8 +227,8 @@ public class TopicsImageTest { TOPIC_RECORD.highestSupportedVersion() ) ); - topicRecords.add(newPartitionRecord(BAM_UUID2, 0, Arrays.asList(0, 1, 2))); - topicRecords.add(newPartitionRecord(BAM_UUID2, 1, Arrays.asList(0, 1, localId))); + topicRecords.add(newPartitionRecord(BAM_UUID2, 0, List.of(0, 1, 2))); + topicRecords.add(newPartitionRecord(BAM_UUID2, 1, List.of(0, 1, localId))); // baz-1 - new partition to follower topicRecords.add( @@ -237,8 +236,8 @@ public class TopicsImageTest { new PartitionRecord() .setPartitionId(1) .setTopicId(BAZ_UUID) - .setReplicas(Arrays.asList(4, 2, localId)) - .setIsr(Arrays.asList(4, 2, localId)) + .setReplicas(List.of(4, 2, localId)) + .setIsr(List.of(4, 2, localId)) .setLeader(4) .setLeaderEpoch(2) .setPartitionEpoch(1), @@ -251,16 +250,16 @@ public class TopicsImageTest { LocalReplicaChanges changes = delta.localChanges(localId); assertEquals( - new HashSet<>(Collections.singletonList(new TopicPartition("baz", 0))), + new HashSet<>(List.of(new TopicPartition("baz", 0))), changes.electedLeaders().keySet() ); assertEquals( - new HashSet<>(Collections.singletonList(new TopicPartition("baz", 0))), + new HashSet<>(List.of(new TopicPartition("baz", 0))), changes.leaders().keySet() ); assertEquals( new HashSet<>( - Arrays.asList(new TopicPartition("baz", 1), new TopicPartition("bar", 0), + List.of(new TopicPartition("baz", 1), new TopicPartition("bar", 0), new TopicPartition("bam", 1)) ), changes.followers().keySet() @@ -307,10 +306,10 @@ public class TopicsImageTest { RecordTestUtils.replayAll(delta, topicRecords); LocalReplicaChanges changes = delta.localChanges(localId); - assertEquals(new HashSet<>(Collections.singletonList(new TopicPartition("zoo", 0))), changes.deletes()); - assertEquals(Collections.emptyMap(), changes.electedLeaders()); - assertEquals(Collections.emptyMap(), changes.leaders()); - assertEquals(Collections.emptyMap(), changes.followers()); + assertEquals(new HashSet<>(List.of(new TopicPartition("zoo", 0))), changes.deletes()); + assertEquals(Map.of(), changes.electedLeaders()); + assertEquals(Map.of(), changes.leaders()); + assertEquals(Map.of(), changes.followers()); TopicsImage finalImage = delta.apply(); List imageRecords = getImageRecords(image); @@ -337,7 +336,7 @@ public class TopicsImageTest { List topicRecords = new ArrayList<>(); topicRecords.add( new ApiMessageAndVersion( - new PartitionChangeRecord().setTopicId(zooId).setPartitionId(0).setIsr(Arrays.asList(localId, 1)), + new PartitionChangeRecord().setTopicId(zooId).setPartitionId(0).setIsr(List.of(localId, 1)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -346,13 +345,13 @@ public class TopicsImageTest { RecordTestUtils.replayAll(delta, topicRecords); LocalReplicaChanges changes = delta.localChanges(localId); - assertEquals(Collections.emptySet(), changes.deletes()); - assertEquals(Collections.emptyMap(), changes.electedLeaders()); + assertEquals(Set.of(), changes.deletes()); + assertEquals(Map.of(), changes.electedLeaders()); assertEquals( - new HashSet<>(Collections.singletonList(new TopicPartition("zoo", 0))), + new HashSet<>(List.of(new TopicPartition("zoo", 0))), changes.leaders().keySet() ); - assertEquals(Collections.emptyMap(), changes.followers()); + assertEquals(Map.of(), changes.followers()); } @Test @@ -375,9 +374,9 @@ public class TopicsImageTest { topicRecords.add( new ApiMessageAndVersion( new PartitionChangeRecord().setTopicId(fooId).setPartitionId(0). - setIsr(Arrays.asList(0, 1)). - setEligibleLeaderReplicas(Arrays.asList(2)). - setLastKnownElr(Arrays.asList(3)), + setIsr(List.of(0, 1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -395,7 +394,7 @@ public class TopicsImageTest { 1. The clear elr record should work on all existing topics(foo). 2. The clear elr record should work on the new topic(bar) in the same batch. */ - topicRecords.addAll(Arrays.asList( + topicRecords.addAll(List.of( new ApiMessageAndVersion( new TopicRecord().setTopicId(barId). setName("bar"), @@ -405,9 +404,9 @@ public class TopicsImageTest { new PartitionRecord().setTopicId(barId). setPartitionId(0). setLeader(0). - setIsr(Arrays.asList(1)). - setEligibleLeaderReplicas(Arrays.asList(2)). - setLastKnownElr(Arrays.asList(3)), + setIsr(List.of(1)). + setEligibleLeaderReplicas(List.of(2)). + setLastKnownElr(List.of(3)), PARTITION_RECORD.highestSupportedVersion() ), new ApiMessageAndVersion( @@ -431,11 +430,11 @@ public class TopicsImageTest { @Test public void testClearElrRecordForNonExistTopic() { - TopicsImage image = new TopicsImage(newTopicsByIdMap(Collections.emptyList()), - newTopicsByNameMap(Collections.emptyList())); + TopicsImage image = new TopicsImage(newTopicsByIdMap(List.of()), + newTopicsByNameMap(List.of())); TopicsDelta delta = new TopicsDelta(image); List topicRecords = new ArrayList<>(); - topicRecords.addAll(Collections.singletonList( + topicRecords.addAll(List.of( new ApiMessageAndVersion( new ClearElrRecord().setTopicName("non-exist"), CLEAR_ELR_RECORD.highestSupportedVersion() @@ -485,8 +484,8 @@ public class TopicsImageTest { new PartitionChangeRecord() .setTopicId(zooId) .setPartitionId(2) - .setIsr(Arrays.asList(0, 1, 2)) - .setReplicas(Arrays.asList(0, 1, 2)), + .setIsr(List.of(0, 1, 2)) + .setReplicas(List.of(0, 1, 2)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -497,8 +496,8 @@ public class TopicsImageTest { .setTopicId(zooId) .setPartitionId(3) .setLeader(0) - .setIsr(Arrays.asList(0, 1, 2)) - .setReplicas(Arrays.asList(0, 1, 2)), + .setIsr(List.of(0, 1, 2)) + .setReplicas(List.of(0, 1, 2)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -509,8 +508,8 @@ public class TopicsImageTest { .setTopicId(zooId) .setPartitionId(4) .setLeader(localId) - .setIsr(Arrays.asList(localId, 1, 2)) - .setReplicas(Arrays.asList(localId, 1, 2)), + .setIsr(List.of(localId, 1, 2)) + .setReplicas(List.of(localId, 1, 2)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -520,8 +519,8 @@ public class TopicsImageTest { new PartitionChangeRecord() .setTopicId(zooId) .setPartitionId(5) - .setIsr(Arrays.asList(0, 1, localId)) - .setReplicas(Arrays.asList(0, 1, localId)), + .setIsr(List.of(0, 1, localId)) + .setReplicas(List.of(0, 1, localId)), PARTITION_CHANGE_RECORD.highestSupportedVersion() ) ); @@ -531,19 +530,19 @@ public class TopicsImageTest { LocalReplicaChanges changes = delta.localChanges(localId); assertEquals( - new HashSet<>(Arrays.asList(new TopicPartition("zoo", 2), new TopicPartition("zoo", 3))), + new HashSet<>(List.of(new TopicPartition("zoo", 2), new TopicPartition("zoo", 3))), changes.deletes() ); assertEquals( - new HashSet<>(Arrays.asList(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))), + new HashSet<>(List.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))), changes.electedLeaders().keySet() ); assertEquals( - new HashSet<>(Arrays.asList(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))), + new HashSet<>(List.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))), changes.leaders().keySet() ); assertEquals( - new HashSet<>(Arrays.asList(new TopicPartition("zoo", 1), new TopicPartition("zoo", 5))), + new HashSet<>(List.of(new TopicPartition("zoo", 1), new TopicPartition("zoo", 5))), changes.followers().keySet() ); @@ -611,7 +610,7 @@ public class TopicsImageTest { assertNull(map.get("baz")); HashSet uuids = new HashSet<>(); map.values().iterator().forEachRemaining(uuids::add); - HashSet expectedUuids = new HashSet<>(Arrays.asList( + HashSet expectedUuids = new HashSet<>(List.of( Uuid.fromString("ThIaNwRnSM2Nt9Mx1v0RvA"), Uuid.fromString("f62ptyETTjet8SL5ZeREiw"))); assertEquals(expectedUuids, uuids); @@ -630,7 +629,7 @@ public class TopicsImageTest { assertNull(map.get(BAZ_UUID)); HashSet names = new HashSet<>(); map.values().iterator().forEachRemaining(names::add); - HashSet expectedNames = new HashSet<>(Arrays.asList("foo", "bar")); + HashSet expectedNames = new HashSet<>(List.of("foo", "bar")); assertEquals(expectedNames, names); assertThrows(UnsupportedOperationException.class, () -> map.remove(FOO_UUID)); assertThrows(UnsupportedOperationException.class, () -> map.put(FOO_UUID, "bar")); diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataBatchLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataBatchLoaderTest.java index e0656c8e267..3df1844ca9c 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataBatchLoaderTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataBatchLoaderTest.java @@ -38,11 +38,8 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.OptionalInt; -import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -65,7 +62,7 @@ public class MetadataBatchLoaderTest { static { { - TOPIC_TXN_BATCH_1 = Arrays.asList( + TOPIC_TXN_BATCH_1 = List.of( new ApiMessageAndVersion(new BeginTransactionRecord().setName("txn-1"), (short) 0), new ApiMessageAndVersion(new TopicRecord() .setName("foo") @@ -75,7 +72,7 @@ public class MetadataBatchLoaderTest { .setTopicId(TOPIC_FOO), (short) 0) ); - TOPIC_TXN_BATCH_2 = Arrays.asList( + TOPIC_TXN_BATCH_2 = List.of( new ApiMessageAndVersion(new PartitionRecord() .setPartitionId(1) .setTopicId(TOPIC_FOO), (short) 0), @@ -85,7 +82,7 @@ public class MetadataBatchLoaderTest { new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0) ); - TOPIC_NO_TXN_BATCH = Arrays.asList( + TOPIC_NO_TXN_BATCH = List.of( new ApiMessageAndVersion(new TopicRecord() .setName("bar") .setTopicId(TOPIC_BAR), (short) 0), @@ -97,13 +94,13 @@ public class MetadataBatchLoaderTest { .setTopicId(TOPIC_BAR), (short) 0) ); - TXN_BEGIN_SINGLETON = Collections.singletonList( + TXN_BEGIN_SINGLETON = List.of( new ApiMessageAndVersion(new BeginTransactionRecord().setName("txn-1"), (short) 0)); - TXN_END_SINGLETON = Collections.singletonList( + TXN_END_SINGLETON = List.of( new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0)); - TXN_ABORT_SINGLETON = Collections.singletonList( + TXN_ABORT_SINGLETON = List.of( new ApiMessageAndVersion(new AbortTransactionRecord(), (short) 0)); } } @@ -111,7 +108,7 @@ public class MetadataBatchLoaderTest { static List noOpRecords(int n) { return IntStream.range(0, n) .mapToObj(__ -> new ApiMessageAndVersion(new NoOpRecord(), (short) 0)) - .collect(Collectors.toList()); + .toList(); } diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java index f1c0a79a202..8c4526de562 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/MetadataLoaderTest.java @@ -50,7 +50,6 @@ import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; -import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Optional; @@ -60,10 +59,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.apache.kafka.server.common.MetadataVersion.IBP_3_5_IV0; import static org.apache.kafka.server.common.MetadataVersion.MINIMUM_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -145,7 +141,7 @@ public class MetadataLoaderTest { @Test public void testInstallPublishers() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testInstallPublishers"); - List publishers = asList(new MockPublisher("a"), + List publishers = List.of(new MockPublisher("a"), new MockPublisher("b"), new MockPublisher("c")); try (MetadataLoader loader = new MetadataLoader.Builder(). @@ -176,7 +172,7 @@ public class MetadataLoaderTest { List> batches = lists .stream() .map(records -> Batch.data(0, 0, 0, 0, records)) - .collect(Collectors.toList()); + .toList(); return new MockSnapshotReader(provenance, batches); } @@ -247,7 +243,7 @@ public class MetadataLoaderTest { setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(0L)). build()) { - loader.installPublishers(singletonList(publisher)).get(); + loader.installPublishers(List.of(publisher)).get(); if (loadSnapshot) { MockSnapshotReader snapshotReader = new MockSnapshotReader( new MetadataProvenance(200, 100, 4000, true), @@ -257,10 +253,10 @@ public class MetadataLoaderTest { 100, 4000, 10, - singletonList(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) + List.of(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) ), Batch.data(0, 0, 0, 0, - singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + List.of(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)) ) @@ -280,13 +276,13 @@ public class MetadataLoaderTest { assertEquals("testPublisherCannotBeInstalledMoreThanOnce: Attempted to install " + "publisher MockPublisher, which is already installed.", assertThrows(ExecutionException.class, - () -> loader.installPublishers(singletonList(publisher)).get()). + () -> loader.installPublishers(List.of(publisher)).get()). getCause().getMessage()); } else { assertEquals("testPublisherCannotBeInstalledMoreThanOnce: Attempted to install " + "a new publisher named MockPublisher, but there is already a publisher with that name.", assertThrows(ExecutionException.class, - () -> loader.installPublishers(singletonList(new MockPublisher())).get()). + () -> loader.installPublishers(List.of(new MockPublisher())).get()). getCause().getMessage()); } } @@ -298,7 +294,7 @@ public class MetadataLoaderTest { @Test public void testRemovePublisher() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testRemovePublisher"); - List publishers = asList(new MockPublisher("a"), + List publishers = List.of(new MockPublisher("a"), new MockPublisher("b"), new MockPublisher("c")); try (MetadataLoader loader = new MetadataLoader.Builder(). @@ -309,7 +305,7 @@ public class MetadataLoaderTest { loader.removeAndClosePublisher(publishers.get(1)).get(); MockSnapshotReader snapshotReader = MockSnapshotReader.fromRecordLists( new MetadataProvenance(100, 50, 2000, true), - singletonList(singletonList(new ApiMessageAndVersion( + List.of(List.of(new ApiMessageAndVersion( new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)))); @@ -337,7 +333,7 @@ public class MetadataLoaderTest { public void testLoadEmptySnapshot() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testLoadEmptySnapshot"); MockTime time = new MockTime(); - List publishers = singletonList(new MockPublisher()); + List publishers = List.of(new MockPublisher()); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setTime(time). @@ -384,13 +380,13 @@ public class MetadataLoaderTest { ) throws Exception { MockSnapshotReader snapshotReader = new MockSnapshotReader( new MetadataProvenance(offset, 100, 4000, true), - singletonList( + List.of( Batch.control( 200, 100, 4000, 10, - singletonList(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) + List.of(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) ) ) ); @@ -413,7 +409,7 @@ public class MetadataLoaderTest { List records ) { return new MockBatchReader(batchBaseOffset, - singletonList(newBatch(batchBaseOffset, epoch, records))); + List.of(newBatch(batchBaseOffset, epoch, records))); } static Batch newBatch( @@ -472,7 +468,7 @@ public class MetadataLoaderTest { public void testLoadEmptyBatch() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testLoadEmptyBatch"); MockTime time = new MockTime(); - List publishers = singletonList(new MockPublisher()); + List publishers = List.of(new MockPublisher()); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setTime(time). @@ -483,13 +479,13 @@ public class MetadataLoaderTest { publishers.get(0).firstPublish.get(10, TimeUnit.SECONDS); MockBatchReader batchReader = new MockBatchReader( 300, - singletonList( + List.of( Batch.control( 300, 100, 4000, 10, - singletonList(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) + List.of(new ControlRecord(ControlRecordType.SNAPSHOT_HEADER, new SnapshotHeaderRecord())) ) ) ).setTime(time); @@ -519,7 +515,7 @@ public class MetadataLoaderTest { @Test public void testLastAppliedOffset() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testLastAppliedOffset"); - List publishers = asList(new MockPublisher("a"), + List publishers = List.of(new MockPublisher("a"), new MockPublisher("b")); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). @@ -527,11 +523,11 @@ public class MetadataLoaderTest { build()) { loader.installPublishers(publishers).get(); loader.handleLoadSnapshot(MockSnapshotReader.fromRecordLists( - new MetadataProvenance(200, 100, 4000, true), asList( - singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + new MetadataProvenance(200, 100, 4000, true), List.of( + List.of(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)), - singletonList(new ApiMessageAndVersion(new TopicRecord(). + List.of(new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) ))); @@ -540,8 +536,8 @@ public class MetadataLoaderTest { } loader.waitForAllEventsToBeHandled(); assertEquals(200L, loader.lastAppliedOffset()); - loader.handleCommit(new MockBatchReader(201, singletonList( - MockBatchReader.newBatch(201, 100, singletonList( + loader.handleCommit(new MockBatchReader(201, List.of( + MockBatchReader.newBatch(201, 100, List.of( new ApiMessageAndVersion(new RemoveTopicRecord(). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)))))); loader.waitForAllEventsToBeHandled(); @@ -562,7 +558,7 @@ public class MetadataLoaderTest { @Test public void testCatchingUpState() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testLastAppliedOffset"); - List publishers = asList(new MockPublisher("a"), + List publishers = List.of(new MockPublisher("a"), new MockPublisher("b")); AtomicReference highWaterMark = new AtomicReference<>(OptionalLong.empty()); try (MetadataLoader loader = new MetadataLoader.Builder(). @@ -596,11 +592,11 @@ public class MetadataLoaderTest { long offset ) throws Exception { loader.handleLoadSnapshot(MockSnapshotReader.fromRecordLists( - new MetadataProvenance(offset, 100, 4000, true), asList( - singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + new MetadataProvenance(offset, 100, 4000, true), List.of( + List.of(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MINIMUM_VERSION.featureLevel()), (short) 0)), - singletonList(new ApiMessageAndVersion(new TopicRecord(). + List.of(new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) ))); @@ -612,11 +608,11 @@ public class MetadataLoaderTest { long offset ) throws Exception { loader.handleLoadSnapshot(MockSnapshotReader.fromRecordLists( - new MetadataProvenance(offset, 100, 4000, true), asList( - singletonList(new ApiMessageAndVersion(new FeatureLevelRecord(). + new MetadataProvenance(offset, 100, 4000, true), List.of( + List.of(new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.latestProduction().featureLevel()), (short) 0)), - singletonList(new ApiMessageAndVersion(new TopicRecord(). + List.of(new ApiMessageAndVersion(new TopicRecord(). setName("bar"). setTopicId(Uuid.fromString("VcL2Mw-cT4aL6XV9VujzoQ")), (short) 0)) ))); @@ -629,7 +625,7 @@ public class MetadataLoaderTest { @Test public void testReloadSnapshot() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testLastAppliedOffset"); - List publishers = singletonList(new MockPublisher("a")); + List publishers = List.of(new MockPublisher("a")); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(0)). @@ -656,8 +652,8 @@ public class MetadataLoaderTest { assertFalse(publishers.get(0).latestImage.topics().topicsByName().containsKey("foo")); assertTrue(publishers.get(0).latestImage.topics().topicsByName().containsKey("bar")); - loader.handleCommit(new MockBatchReader(500, singletonList( - MockBatchReader.newBatch(500, 100, singletonList( + loader.handleCommit(new MockBatchReader(500, List.of( + MockBatchReader.newBatch(500, 100, List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(IBP_3_5_IV0.featureLevel()), (short) 0)))))); @@ -673,7 +669,7 @@ public class MetadataLoaderTest { public void testPublishTransaction(boolean abortTxn) throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testTransactions"); MockPublisher publisher = new MockPublisher("testTransactions"); - List publishers = singletonList(publisher); + List publishers = List.of(publisher); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(0)). @@ -688,7 +684,7 @@ public class MetadataLoaderTest { loader.waitForAllEventsToBeHandled(); loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, Arrays.asList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new BeginTransactionRecord() .setName("testTransactions"), (short) 0), new ApiMessageAndVersion(new TopicRecord() @@ -701,7 +697,7 @@ public class MetadataLoaderTest { "Topic should not be visible since we started transaction"); loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, Arrays.asList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new PartitionRecord() .setTopicId(Uuid.fromString("dMCqhcK4T5miGH5wEX7NsQ")) .setPartitionId(0), (short) 0), @@ -715,7 +711,7 @@ public class MetadataLoaderTest { if (abortTxn) { loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, singletonList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new AbortTransactionRecord(), (short) 0) ))); loader.waitForAllEventsToBeHandled(); @@ -724,7 +720,7 @@ public class MetadataLoaderTest { "Topic should not be visible since the transaction was aborted"); } else { loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, singletonList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new EndTransactionRecord(), (short) 0) ))); loader.waitForAllEventsToBeHandled(); @@ -740,7 +736,7 @@ public class MetadataLoaderTest { public void testPublishTransactionWithinBatch() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testPublishTransactionWithinBatch"); MockPublisher publisher = new MockPublisher("testPublishTransactionWithinBatch"); - List publishers = singletonList(publisher); + List publishers = List.of(publisher); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(0)). @@ -755,7 +751,7 @@ public class MetadataLoaderTest { loader.waitForAllEventsToBeHandled(); loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, Arrays.asList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new BeginTransactionRecord() .setName("txn-1"), (short) 0), new ApiMessageAndVersion(new TopicRecord() @@ -777,7 +773,7 @@ public class MetadataLoaderTest { public void testSnapshotDuringTransaction() throws Exception { MockFaultHandler faultHandler = new MockFaultHandler("testSnapshotDuringTransaction"); MockPublisher publisher = new MockPublisher("testSnapshotDuringTransaction"); - List publishers = singletonList(publisher); + List publishers = List.of(publisher); try (MetadataLoader loader = new MetadataLoader.Builder(). setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(0)). @@ -792,7 +788,7 @@ public class MetadataLoaderTest { loader.waitForAllEventsToBeHandled(); loader.handleCommit( - MockBatchReader.newSingleBatchReader(500, 100, Arrays.asList( + MockBatchReader.newSingleBatchReader(500, 100, List.of( new ApiMessageAndVersion(new BeginTransactionRecord() .setName("txn-1"), (short) 0), new ApiMessageAndVersion(new TopicRecord() @@ -805,8 +801,8 @@ public class MetadataLoaderTest { // loading a snapshot discards any in-flight transaction loader.handleLoadSnapshot(MockSnapshotReader.fromRecordLists( - new MetadataProvenance(600, 101, 4000, true), singletonList( - singletonList(new ApiMessageAndVersion(new TopicRecord(). + new MetadataProvenance(600, 101, 4000, true), List.of( + List.of(new ApiMessageAndVersion(new TopicRecord(). setName("foo"). setTopicId(Uuid.fromString("Uum7sfhHQP-obSvfywmNUA")), (short) 0)) ))); @@ -841,9 +837,9 @@ public class MetadataLoaderTest { setFaultHandler(faultHandler). setHighWaterMarkAccessor(() -> OptionalLong.of(1)). build()) { - loader.installPublishers(singletonList(capturingPublisher)).get(); + loader.installPublishers(List.of(capturingPublisher)).get(); loader.handleCommit( - MockBatchReader.newSingleBatchReader(0, 1, singletonList( + MockBatchReader.newSingleBatchReader(0, 1, List.of( // Any record will work here new ApiMessageAndVersion(new FeatureLevelRecord() .setName(MetadataVersion.FEATURE_NAME) diff --git a/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java b/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java index 13062c98169..02ee90044aa 100644 --- a/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/loader/metrics/MetadataLoaderMetricsTest.java @@ -26,10 +26,10 @@ import com.yammer.metrics.core.MetricsRegistry; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -70,14 +70,14 @@ public class MetadataLoaderMetricsTest { try { try (FakeMetadataLoaderMetrics fakeMetrics = new FakeMetadataLoaderMetrics(registry)) { ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.server", - new HashSet<>(Arrays.asList( + new HashSet<>(List.of( "kafka.server:type=MetadataLoader,name=CurrentControllerId", "kafka.server:type=MetadataLoader,name=CurrentMetadataVersion", "kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount" ))); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.server", - Collections.emptySet()); + Set.of()); } finally { registry.shutdown(); } @@ -134,7 +134,7 @@ public class MetadataLoaderMetricsTest { assertEquals(2L, loadSnapshotCount.value().longValue()); } ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.server", - Collections.emptySet()); + Set.of()); } finally { registry.shutdown(); } diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ClientQuotasImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ClientQuotasImageNodeTest.java index c22ae8821a4..e273b4a2c91 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ClientQuotasImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ClientQuotasImageNodeTest.java @@ -25,8 +25,6 @@ import org.junit.jupiter.api.Timeout; import java.util.HashMap; import java.util.Map; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -62,25 +60,25 @@ public class ClientQuotasImageNodeTest { @Test public void clientIdEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("client-id", "foo")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("client-id", "foo")), "clientId(foo)"); } @Test public void defaultClientIdEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("client-id", "")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("client-id", "")), "clientId()"); } @Test public void userEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("user", "my-user-name")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("user", "my-user-name")), "user(my-user-name)"); } @Test public void defaultUserEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("user", "")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("user", "")), "user()"); } @@ -95,25 +93,25 @@ public class ClientQuotasImageNodeTest { @Test public void ipEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("ip", "127.0.0.1")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("ip", "127.0.0.1")), "ip(127.0.0.1)"); } @Test public void defaultIpEntityRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("ip", "")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("ip", "")), "ip()"); } @Test public void testUserEntityWithBackslashesInNameRoundTrip() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("user", "foo\\bar")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("user", "foo\\bar")), "user(foo\\\\bar)"); } @Test public void testClientIdEntityWithParentheses() { - entityToStringRoundTrip(new ClientQuotaEntity(singletonMap("client-id", "(this )one)")), + entityToStringRoundTrip(new ClientQuotaEntity(Map.of("client-id", "(this )one)")), "clientId(\\(this \\)one\\))"); } @@ -121,7 +119,7 @@ public class ClientQuotasImageNodeTest { public void testErrorOnInvalidEmptyEntityName() { assertEquals("Invalid empty entity", assertThrows(RuntimeException.class, () -> ClientQuotasImageNode. - clientQuotaEntityToString(new ClientQuotaEntity(emptyMap()))). + clientQuotaEntityToString(new ClientQuotaEntity(Map.of()))). getMessage()); } @@ -129,7 +127,7 @@ public class ClientQuotasImageNodeTest { public void testErrorOnInvalidEntityType() { assertEquals("Invalid entity type foobar", assertThrows(RuntimeException.class, () -> ClientQuotasImageNode. - clientQuotaEntityToString(new ClientQuotaEntity(singletonMap("foobar", "baz")))). + clientQuotaEntityToString(new ClientQuotaEntity(Map.of("foobar", "baz")))). getMessage()); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageBrokersNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageBrokersNodeTest.java index e846443d0fb..6d7b8898810 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageBrokersNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageBrokersNodeTest.java @@ -26,8 +26,8 @@ import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -38,23 +38,23 @@ import static org.junit.jupiter.api.Assertions.assertNull; @Timeout(value = 40) public class ClusterImageBrokersNodeTest { private static final ClusterImage TEST_IMAGE = new ClusterImage( - Collections.singletonMap(1, new BrokerRegistration.Builder(). + Map.of(1, new BrokerRegistration.Builder(). setId(1). setEpoch(1001). setIncarnationId(Uuid.fromString("MJkaH0j0RwuC3W2GHQHtWA")). - setListeners(Collections.emptyList()). - setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 4))). + setListeners(List.of()). + setSupportedFeatures(Map.of(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 4))). setRack(Optional.empty()). setFenced(false). - setDirectories(Arrays.asList(Uuid.fromString("anCdBWcFTlu8gE1wP6bh3g"), Uuid.fromString("JsnDDNVyTL289kYk6sPzig"))). + setDirectories(List.of(Uuid.fromString("anCdBWcFTlu8gE1wP6bh3g"), Uuid.fromString("JsnDDNVyTL289kYk6sPzig"))). setInControlledShutdown(false).build()), - Collections.emptyMap()); + Map.of()); private static final ClusterImageBrokersNode NODE = new ClusterImageBrokersNode(TEST_IMAGE); @Test public void testChildNames() { - assertEquals(Collections.singletonList("1"), NODE.childNames()); + assertEquals(List.of("1"), NODE.childNames()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageControllersNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageControllersNodeTest.java index 43f52dabaa2..f017a172357 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageControllersNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageControllersNodeTest.java @@ -26,7 +26,8 @@ import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; +import java.util.List; +import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -36,21 +37,21 @@ import static org.junit.jupiter.api.Assertions.assertNull; @Timeout(value = 40) public class ClusterImageControllersNodeTest { private static final ClusterImage TEST_IMAGE = new ClusterImage( - Collections.emptyMap(), - Collections.singletonMap(2, new ControllerRegistration.Builder(). + Map.of(), + Map.of(2, new ControllerRegistration.Builder(). setId(2). setIncarnationId(Uuid.fromString("adGo6sTPS0uJshjvdTUmqQ")). setZkMigrationReady(false). - setSupportedFeatures(Collections.singletonMap( + setSupportedFeatures(Map.of( MetadataVersion.FEATURE_NAME, VersionRange.of(1, 4))). - setListeners(Collections.emptyMap()). + setListeners(Map.of()). build())); private static final ClusterImageControllersNode NODE = new ClusterImageControllersNode(TEST_IMAGE); @Test public void testChildNames() { - assertEquals(Collections.singletonList("2"), NODE.childNames()); + assertEquals(List.of("2"), NODE.childNames()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageNodeTest.java index 1c9e6c81927..2f48d4ee5de 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ClusterImageNodeTest.java @@ -22,7 +22,7 @@ import org.apache.kafka.image.ClusterImage; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -35,7 +35,7 @@ public class ClusterImageNodeTest { @Test public void testChildNames() { - assertEquals(Arrays.asList("brokers", "controllers"), NODE.childNames()); + assertEquals(List.of("brokers", "controllers"), NODE.childNames()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationImageNodeTest.java index d7a19d034f7..b335a1be01b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationImageNodeTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.metadata.KafkaConfigSchema; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -43,11 +42,11 @@ public class ConfigurationImageNodeTest { private static final ConfigurationImageNode NODE; static { - KafkaConfigSchema schema = new KafkaConfigSchema(Collections.singletonMap(BROKER, new ConfigDef(). + KafkaConfigSchema schema = new KafkaConfigSchema(Map.of(BROKER, new ConfigDef(). define("non.secret", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "baz"). define("also.non.secret", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "baz"). define("secret.config", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "baz")), - Collections.emptyMap()); + Map.of()); NORMAL = new MetadataNodeRedactionCriteria.Normal(schema); Map configs = new HashMap<>(); diff --git a/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationsImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationsImageNodeTest.java index f34e86e191d..e133c0384b2 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationsImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/ConfigurationsImageNodeTest.java @@ -25,12 +25,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; @@ -43,7 +40,7 @@ public class ConfigurationsImageNodeTest { static { Map resourceMap = new HashMap<>(); - for (ConfigResource resource : Arrays.asList( + for (ConfigResource resource : List.of( new ConfigResource(BROKER, ""), new ConfigResource(BROKER, "0"), new ConfigResource(TOPIC, ""), @@ -51,7 +48,7 @@ public class ConfigurationsImageNodeTest { new ConfigResource(TOPIC, ":colons:"), new ConfigResource(TOPIC, "__internal"))) { resourceMap.put(resource, new ConfigurationImage(resource, - Collections.singletonMap("foo", "bar"))); + Map.of("foo", "bar"))); } ConfigurationsImage image = new ConfigurationsImage(resourceMap); NODE = new ConfigurationsImageNode(image); @@ -61,7 +58,7 @@ public class ConfigurationsImageNodeTest { public void testNodeChildNames() { List childNames = new ArrayList<>(NODE.childNames()); childNames.sort(String::compareTo); - assertEquals(Arrays.asList( + assertEquals(List.of( "BROKER", "BROKER:0", "TOPIC", @@ -75,8 +72,8 @@ public class ConfigurationsImageNodeTest { List childResources = NODE.childNames().stream(). sorted(). map(ConfigurationsImageNode::resourceFromName). - collect(Collectors.toList()); - assertEquals(Arrays.asList( + toList(); + assertEquals(List.of( new ConfigResource(BROKER, ""), new ConfigResource(BROKER, "0"), new ConfigResource(TOPIC, ""), diff --git a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java index 60d4587c340..84d70cd90c1 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/TopicImageNodeTest.java @@ -27,8 +27,8 @@ import org.apache.kafka.metadata.PartitionRegistration; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -52,7 +52,7 @@ public class TopicImageNodeTest { } @Test public void testChildNames() { - assertEquals(Arrays.asList("name", "id", "0"), NODE.childNames()); + assertEquals(List.of("name", "id", "0"), NODE.childNames()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/image/node/printer/MetadataNodeRedactionCriteriaTest.java b/metadata/src/test/java/org/apache/kafka/image/node/printer/MetadataNodeRedactionCriteriaTest.java index a6d1a65ed7d..32d3f3cc55b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/node/printer/MetadataNodeRedactionCriteriaTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/node/printer/MetadataNodeRedactionCriteriaTest.java @@ -23,7 +23,6 @@ import org.apache.kafka.metadata.KafkaConfigSchema; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -47,7 +46,7 @@ public class MetadataNodeRedactionCriteriaTest { define("secret.config", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "baz")); configs.put(TOPIC, new ConfigDef(). define("topic.secret.config", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "abc")); - KafkaConfigSchema schema = new KafkaConfigSchema(configs, Collections.emptyMap()); + KafkaConfigSchema schema = new KafkaConfigSchema(configs, Map.of()); STRICT = MetadataNodeRedactionCriteria.Strict.INSTANCE; NORMAL = new MetadataNodeRedactionCriteria.Normal(schema); DISABLED = MetadataNodeRedactionCriteria.Disabled.INSTANCE; diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java index 5c3bc5a1230..2b90702b517 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/BrokerRegistrationTrackerTest.java @@ -32,8 +32,7 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -98,7 +97,7 @@ public class BrokerRegistrationTrackerTest { delta.replay(new RegisterBrokerRecord(). setBrokerId(1). setIncarnationId(INCARNATION_ID). - setLogDirs(Arrays.asList(A, B, C))); + setLogDirs(List.of(A, B, C))); delta.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.MINIMUM_VERSION.featureLevel())); @@ -114,7 +113,7 @@ public class BrokerRegistrationTrackerTest { delta.replay(new RegisterBrokerRecord(). setBrokerId(1). setIncarnationId(INCARNATION_ID). - setLogDirs(Collections.emptyList())); + setLogDirs(List.of())); delta.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(jbodMv ? MetadataVersion.IBP_3_7_IV2.featureLevel() : @@ -135,7 +134,7 @@ public class BrokerRegistrationTrackerTest { delta.replay(new RegisterBrokerRecord(). setBrokerId(1). setIncarnationId(INCARNATION_ID). - setLogDirs(Collections.emptyList())); + setLogDirs(List.of())); delta.replay(new FeatureLevelRecord(). setName(MetadataVersion.FEATURE_NAME). setFeatureLevel(MetadataVersion.IBP_3_7_IV1.featureLevel())); diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisherTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisherTest.java index 4d47580313f..b97643d4a3b 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisherTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/ControllerRegistrationsPublisherTest.java @@ -32,9 +32,9 @@ import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.OptionalInt; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -47,7 +47,7 @@ public class ControllerRegistrationsPublisherTest { @Test public void testInitialControllers() { ControllerRegistrationsPublisher publisher = new ControllerRegistrationsPublisher(); - assertEquals(Collections.emptyMap(), publisher.controllers()); + assertEquals(Map.of(), publisher.controllers()); } @Test @@ -91,7 +91,7 @@ public class ControllerRegistrationsPublisherTest { build()); } System.out.println("TEST_IMAGE.cluster = " + TEST_IMAGE.cluster()); - assertEquals(new HashSet<>(Arrays.asList(0, 1, 2)), publisher.controllers().keySet()); + assertEquals(new HashSet<>(List.of(0, 1, 2)), publisher.controllers().keySet()); assertTrue(publisher.controllers().get(0).zkMigrationReady()); assertFalse(publisher.controllers().get(1).zkMigrationReady()); assertFalse(publisher.controllers().get(2).zkMigrationReady()); diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java index 313c2bc14e6..db3c2e0ad16 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/SnapshotGeneratorTest.java @@ -32,7 +32,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -124,10 +123,10 @@ public class SnapshotGeneratorTest { // Publish a log delta batch. This one will be ignored because there are other images // queued for writing. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, logDeltaManifestBuilder().numBytes(2000).build()); - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); emitter.setReady(); } - assertEquals(Collections.singletonList(TEST_IMAGE), emitter.images()); + assertEquals(List.of(TEST_IMAGE), emitter.images()); faultHandler.maybeRethrowFirstException(); } @@ -144,10 +143,10 @@ public class SnapshotGeneratorTest { generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, notBatchAlignedLogDeltaManifestBuilder().build()); generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, notBatchAlignedLogDeltaManifestBuilder().build()); generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, notBatchAlignedLogDeltaManifestBuilder().build()); - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); emitter.setReady(); } - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); faultHandler.maybeRethrowFirstException(); } @@ -167,10 +166,10 @@ public class SnapshotGeneratorTest { generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, notBatchAlignedLogDeltaManifestBuilder().build()); // This snapshot should get published since it is batch aligned. generator.publishLogDelta(TEST_DELTA, batchAlignedImage, logDeltaManifestBuilder().build()); - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); emitter.setReady(); } - assertEquals(Collections.singletonList(batchAlignedImage), emitter.images()); + assertEquals(List.of(batchAlignedImage), emitter.images()); faultHandler.maybeRethrowFirstException(); } @@ -189,7 +188,7 @@ public class SnapshotGeneratorTest { // No snapshots are generated because snapshots are disabled. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, logDeltaManifestBuilder().build()); } - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); faultHandler.maybeRethrowFirstException(); } @@ -206,7 +205,7 @@ public class SnapshotGeneratorTest { build()) { // This image isn't published yet. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, logDeltaManifestBuilder().numBytes(50).build()); - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); mockTime.sleep(TimeUnit.MINUTES.toNanos(40)); // Next image is published because of the time delay. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, logDeltaManifestBuilder().numBytes(50).build()); @@ -215,7 +214,7 @@ public class SnapshotGeneratorTest { // so this does not trigger a new snapshot. generator.publishLogDelta(TEST_DELTA, TEST_IMAGE, logDeltaManifestBuilder().numBytes(150).build()); } - assertEquals(Collections.singletonList(TEST_IMAGE), emitter.images()); + assertEquals(List.of(TEST_IMAGE), emitter.images()); faultHandler.maybeRethrowFirstException(); } @@ -232,7 +231,7 @@ public class SnapshotGeneratorTest { logDeltaManifestBuilder().elapsedNs(10000).numBytes(50000).build()); } } - assertEquals(Collections.emptyList(), emitter.images()); + assertEquals(List.of(), emitter.images()); assertNotNull(faultHandler.firstException()); assertEquals(FaultHandlerException.class, faultHandler.firstException().getClass()); assertEquals("SnapshotGenerator: KRaft snapshot file generation error: oops", diff --git a/metadata/src/test/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetricsTest.java b/metadata/src/test/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetricsTest.java index ed6130c0c33..a9ed94a8f66 100644 --- a/metadata/src/test/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetricsTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/publisher/metrics/SnapshotEmitterMetricsTest.java @@ -29,10 +29,10 @@ import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Optional; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -78,13 +78,13 @@ public class SnapshotEmitterMetricsTest { public void testMetricNames() { try (SnapshotEmitterMetricsTestContext ctx = new SnapshotEmitterMetricsTestContext()) { ControllerMetricsTestUtils.assertMetricsForTypeEqual(ctx.registry, "kafka.server:", - new HashSet<>(Arrays.asList( + new HashSet<>(List.of( "kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes", "kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs" ))); ctx.metrics.close(); ControllerMetricsTestUtils.assertMetricsForTypeEqual(ctx.registry, "KafkaController", - Collections.emptySet()); + Set.of()); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java index c089fd77d3f..d949e30c090 100644 --- a/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/writer/ImageReWriterTest.java @@ -27,7 +27,7 @@ import org.apache.kafka.image.MetadataImage; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; +import java.util.Map; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.metadata.RecordTestUtils.testRecord; @@ -87,8 +87,8 @@ public class ImageReWriterTest { writer2.close(true); MetadataImage newImage = writer2.image(); - assertEquals(Collections.emptyMap(), newImage.topics().topicsById()); - assertEquals(Collections.singletonMap("num.io.threads", "12"), + assertEquals(Map.of(), newImage.topics().topicsById()); + assertEquals(Map.of("num.io.threads", "12"), newImage.configs().configMapForResource(new ConfigResource(BROKER, ""))); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java index 3602e8988b8..21d4378c733 100644 --- a/metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/writer/RaftSnapshotWriterTest.java @@ -22,9 +22,8 @@ import org.apache.kafka.image.FakeSnapshotWriter; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; +import java.util.List; -import static java.util.Collections.emptyList; import static org.apache.kafka.metadata.RecordTestUtils.testRecord; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -43,9 +42,9 @@ public class RaftSnapshotWriterTest { writer.close(true); assertTrue(snapshotWriter.isFrozen()); assertTrue(snapshotWriter.isClosed()); - assertEquals(Arrays.asList( - Arrays.asList(testRecord(0), testRecord(1)), - Arrays.asList(testRecord(2))), snapshotWriter.batches()); + assertEquals(List.of( + List.of(testRecord(0), testRecord(1)), + List.of(testRecord(2))), snapshotWriter.batches()); } @Test @@ -56,6 +55,6 @@ public class RaftSnapshotWriterTest { writer.close(); assertFalse(snapshotWriter.isFrozen()); assertTrue(snapshotWriter.isClosed()); - assertEquals(emptyList(), snapshotWriter.batches()); + assertEquals(List.of(), snapshotWriter.batches()); } } diff --git a/metadata/src/test/java/org/apache/kafka/image/writer/RecordListWriterTest.java b/metadata/src/test/java/org/apache/kafka/image/writer/RecordListWriterTest.java index a3b5627f3a3..87cdc685d04 100644 --- a/metadata/src/test/java/org/apache/kafka/image/writer/RecordListWriterTest.java +++ b/metadata/src/test/java/org/apache/kafka/image/writer/RecordListWriterTest.java @@ -23,9 +23,8 @@ import org.apache.kafka.common.metadata.TopicRecord; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; +import java.util.List; -import static java.util.Collections.emptyList; import static org.apache.kafka.metadata.RecordTestUtils.testRecord; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -39,7 +38,7 @@ public class RecordListWriterTest { writer.write(testRecord(0)); writer.write(testRecord(1)); writer.close(true); - assertEquals(Arrays.asList(testRecord(0), testRecord(1)), writer.records()); + assertEquals(List.of(testRecord(0), testRecord(1)), writer.records()); } @Test @@ -47,7 +46,7 @@ public class RecordListWriterTest { RecordListWriter writer = new RecordListWriter(); writer.write(testRecord(0)); writer.close(); - assertEquals(emptyList(), writer.records()); + assertEquals(List.of(), writer.records()); } @Test diff --git a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java index f415e506055..49ac33490b0 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/BrokerRegistrationTest.java @@ -31,9 +31,8 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.AbstractMap.SimpleEntry; -import java.util.Arrays; -import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -45,13 +44,13 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(value = 40) public class BrokerRegistrationTest { - private static final List REGISTRATIONS = Arrays.asList( + private static final List REGISTRATIONS = List.of( new BrokerRegistration.Builder(). setId(0). setEpoch(0). setIncarnationId(Uuid.fromString("pc1GhUlBS92cGGaKXl6ipw")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 2))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(false).build(), @@ -59,8 +58,8 @@ public class BrokerRegistrationTest { setId(1). setEpoch(0). setIncarnationId(Uuid.fromString("3MfdxWlNSn2UDYsmDP1pYg")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9091))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9091))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 2))). setRack(Optional.empty()). setFenced(true). setInControlledShutdown(false).build(), @@ -68,7 +67,7 @@ public class BrokerRegistrationTest { setId(2). setEpoch(0). setIncarnationId(Uuid.fromString("eY7oaG1RREie5Kk9uy1l6g")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9092))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9092))). setSupportedFeatures(Stream.of(new SimpleEntry<>("foo", VersionRange.of((short) 2, (short) 3)), new SimpleEntry<>("bar", VersionRange.of((short) 1, (short) 4))).collect( Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue))). @@ -79,14 +78,14 @@ public class BrokerRegistrationTest { setId(3). setEpoch(0). setIncarnationId(Uuid.fromString("1t8VyWx2TCSTpUWuqj-FOw")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093))). setSupportedFeatures(Stream.of(new SimpleEntry<>("metadata.version", VersionRange.of((short) 7, (short) 7))) .collect(Collectors.toMap(SimpleEntry::getKey, SimpleEntry::getValue))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(true). setIsMigratingZkBroker(true). - setDirectories(Collections.singletonList(Uuid.fromString("r4HpEsMuST6nQ4rznIEJVA"))). + setDirectories(List.of(Uuid.fromString("r4HpEsMuST6nQ4rznIEJVA"))). build()); @Test @@ -174,12 +173,12 @@ public class BrokerRegistrationTest { setId(0). setEpoch(0). setIncarnationId(Uuid.fromString("ik32HZbLTW6ulw1yyrC8jQ")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 2))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(false). - setDirectories(Arrays.asList( + setDirectories(List.of( Uuid.fromString("3MWIBL9NR4eXhtdfBVA7Bw"), Uuid.fromString("SZQIVeLMQGiNi68StNSNZA"), Uuid.fromString("LWZsWPBrQruOMMrnEBj7bw"), @@ -188,7 +187,7 @@ public class BrokerRegistrationTest { Uuid.fromString("VNetSHnySxSbvjwKrBzpkw")) ). build(); - assertEquals(Arrays.asList( + assertEquals(List.of( Uuid.fromString("3MWIBL9NR4eXhtdfBVA7Bw"), Uuid.fromString("JhYia5HRTLihf2FFJVxopQ"), Uuid.fromString("LWZsWPBrQruOMMrnEBj7bw"), @@ -204,12 +203,12 @@ public class BrokerRegistrationTest { setId(0). setEpoch(0). setIncarnationId(Uuid.fromString("m6CiJvfITZeKVC6UuhlZew")). - setListeners(Collections.singletonList(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). - setSupportedFeatures(Collections.singletonMap("foo", VersionRange.of((short) 1, (short) 2))). + setListeners(List.of(new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9090))). + setSupportedFeatures(Map.of("foo", VersionRange.of((short) 1, (short) 2))). setRack(Optional.empty()). setFenced(false). setInControlledShutdown(false). - setDirectories(Arrays.asList( + setDirectories(List.of( Uuid.fromString("dir1G6EtuR1OTdAzFw1AFQ"), Uuid.fromString("dir2gwpjTvKC7sMfcLNd8g"), Uuid.fromString("dir3Ir8mQ0mMxfv93RITDA") diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java index a24de3092bb..2e0e0e5f07b 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ControllerRegistrationTest.java @@ -29,9 +29,7 @@ import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -43,13 +41,13 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; @Timeout(value = 40) public class ControllerRegistrationTest { static Map doubleMap(K k1, V v1, K k2, V v2) { - HashMap map = new HashMap<>(); + LinkedHashMap map = new LinkedHashMap<>(); map.put(k1, v1); map.put(k2, v2); - return Collections.unmodifiableMap(map); + return map; } - private static final List REGISTRATIONS = Arrays.asList( + private static final List REGISTRATIONS = List.of( new ControllerRegistration.Builder(). setId(0). setIncarnationId(Uuid.fromString("ycRmGrOFQru7HXf6fOybZQ")). @@ -57,7 +55,7 @@ public class ControllerRegistrationTest { setListeners(doubleMap( "PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9107), "SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9207))). - setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). + setSupportedFeatures(Map.of(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). build(), new ControllerRegistration.Builder(). setId(1). @@ -66,7 +64,7 @@ public class ControllerRegistrationTest { setListeners(doubleMap( "PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9108), "SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9208))). - setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). + setSupportedFeatures(Map.of(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). build(), new ControllerRegistration.Builder(). setId(2). @@ -75,7 +73,7 @@ public class ControllerRegistrationTest { setListeners(doubleMap( "PLAINTEXT", new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "localhost", 9109), "SSL", new Endpoint("SSL", SecurityProtocol.SSL, "localhost", 9209))). - setSupportedFeatures(Collections.singletonMap(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). + setSupportedFeatures(Map.of(MetadataVersion.FEATURE_NAME, VersionRange.of(1, 10))). build() ); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/DelegationTokenDataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/DelegationTokenDataTest.java index f945b373be1..68ab279042d 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/DelegationTokenDataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/DelegationTokenDataTest.java @@ -26,8 +26,6 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -36,14 +34,14 @@ import static org.junit.jupiter.api.Assertions.assertNotEquals; @Timeout(value = 40) public class DelegationTokenDataTest { - private static final List UUID = Arrays.asList( + private static final List UUID = List.of( Uuid.randomUuid().toString(), Uuid.randomUuid().toString(), Uuid.randomUuid().toString()); - private static final List EMPTYRENEWERS = Collections.emptyList(); + private static final List EMPTYRENEWERS = List.of(); - private static final List TOKENINFORMATION = Arrays.asList( + private static final List TOKENINFORMATION = List.of( new TokenInformation( UUID.get(0), new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "alice"), @@ -69,7 +67,7 @@ public class DelegationTokenDataTest { 100, 100)); - private static final List DELEGATIONTOKENDATA = Arrays.asList( + private static final List DELEGATIONTOKENDATA = List.of( new DelegationTokenData(TOKENINFORMATION.get(0)), new DelegationTokenData(TOKENINFORMATION.get(1)), new DelegationTokenData(TOKENINFORMATION.get(2))); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java b/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java index c535e108878..e1e315e7de3 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/FakeKafkaConfigSchema.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.server.config.ConfigSynonym; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -51,9 +50,9 @@ public class FakeKafkaConfigSchema { static { SYNONYMS.put("unclean.leader.election.enable", - Arrays.asList(new ConfigSynonym("unclean.leader.election.enable"))); + List.of(new ConfigSynonym("unclean.leader.election.enable"))); SYNONYMS.put("min.insync.replicas", - Arrays.asList(new ConfigSynonym("min.insync.replicas"))); + List.of(new ConfigSynonym("min.insync.replicas"))); } public static final KafkaConfigSchema INSTANCE = new KafkaConfigSchema(CONFIGS, SYNONYMS); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java index e498d0b9634..27683ce8933 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/KafkaConfigSchemaTest.java @@ -26,12 +26,10 @@ import org.apache.kafka.server.config.ConfigSynonym; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import static java.util.Collections.emptyList; import static org.apache.kafka.common.config.ConfigResource.Type.BROKER; import static org.apache.kafka.common.config.ConfigResource.Type.TOPIC; import static org.apache.kafka.server.config.ConfigSynonym.HOURS_TO_MILLISECONDS; @@ -62,10 +60,10 @@ public class KafkaConfigSchemaTest { public static final Map> SYNONYMS = new HashMap<>(); static { - SYNONYMS.put("abc", Arrays.asList(new ConfigSynonym("foo.bar"))); - SYNONYMS.put("def", Arrays.asList(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS))); - SYNONYMS.put("ghi", Arrays.asList(new ConfigSynonym("ghi"))); - SYNONYMS.put("xyz", Arrays.asList(new ConfigSynonym("quuux"), new ConfigSynonym("quuux2"))); + SYNONYMS.put("abc", List.of(new ConfigSynonym("foo.bar"))); + SYNONYMS.put("def", List.of(new ConfigSynonym("quux", HOURS_TO_MILLISECONDS))); + SYNONYMS.put("ghi", List.of(new ConfigSynonym("ghi"))); + SYNONYMS.put("xyz", List.of(new ConfigSynonym("quuux"), new ConfigSynonym("quuux2"))); } private static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, SYNONYMS); @@ -153,16 +151,16 @@ public class KafkaConfigSchemaTest { dynamicTopicConfigs.put("ghi", "true"); Map expected = new HashMap<>(); expected.put("abc", new ConfigEntry("abc", "the,dynamic,cluster,config,value", - ConfigEntry.ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, false, false, emptyList(), + ConfigEntry.ConfigSource.DYNAMIC_DEFAULT_BROKER_CONFIG, false, false, List.of(), ConfigEntry.ConfigType.LIST, "abc doc")); expected.put("def", new ConfigEntry("def", "2840400000", - ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, emptyList(), + ConfigEntry.ConfigSource.DYNAMIC_BROKER_CONFIG, false, false, List.of(), ConfigEntry.ConfigType.LONG, "def doc")); expected.put("ghi", new ConfigEntry("ghi", "true", - ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, emptyList(), + ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, List.of(), ConfigEntry.ConfigType.BOOLEAN, "ghi doc")); expected.put("xyz", new ConfigEntry("xyz", "thedefault", - ConfigEntry.ConfigSource.DEFAULT_CONFIG, true, false, emptyList(), + ConfigEntry.ConfigSource.DEFAULT_CONFIG, true, false, List.of(), ConfigEntry.ConfigType.PASSWORD, "xyz doc")); assertEquals(expected, SCHEMA.resolveEffectiveTopicConfigs(staticNodeConfig, dynamicClusterConfigs, diff --git a/metadata/src/test/java/org/apache/kafka/metadata/LeaderAndIsrTest.java b/metadata/src/test/java/org/apache/kafka/metadata/LeaderAndIsrTest.java index 76bc86866df..f6d7b66fba3 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/LeaderAndIsrTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/LeaderAndIsrTest.java @@ -19,48 +19,47 @@ package org.apache.kafka.metadata; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; public final class LeaderAndIsrTest { @Test public void testRecoveringLeaderAndIsr() { - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, Arrays.asList(1, 2)); - LeaderAndIsr recoveringLeaderAndIsr = leaderAndIsr.newRecoveringLeaderAndIsr(3, Collections.singletonList(3)); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, List.of(1, 2)); + LeaderAndIsr recoveringLeaderAndIsr = leaderAndIsr.newRecoveringLeaderAndIsr(3, List.of(3)); assertEquals(3, recoveringLeaderAndIsr.leader()); - assertEquals(Collections.singletonList(3), recoveringLeaderAndIsr.isr()); + assertEquals(List.of(3), recoveringLeaderAndIsr.isr()); assertEquals(LeaderRecoveryState.RECOVERING, recoveringLeaderAndIsr.leaderRecoveryState()); } @Test public void testNewLeaderAndIsr() { - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, Arrays.asList(1, 2)); - LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(2, Arrays.asList(1, 2)); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, List.of(1, 2)); + LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(2, List.of(1, 2)); assertEquals(2, newLeaderAndIsr.leader()); - assertEquals(Arrays.asList(1, 2), newLeaderAndIsr.isr()); + assertEquals(List.of(1, 2), newLeaderAndIsr.isr()); assertEquals(LeaderRecoveryState.RECOVERED, newLeaderAndIsr.leaderRecoveryState()); } @Test public void testNewLeader() { - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, Arrays.asList(1, 2, 3)); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, List.of(1, 2, 3)); assertEquals(2, leaderAndIsr.leader()); - assertEquals(Arrays.asList(1, 2, 3), leaderAndIsr.isr()); + assertEquals(List.of(1, 2, 3), leaderAndIsr.isr()); LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeader(3); assertEquals(3, newLeaderAndIsr.leader()); - assertEquals(Arrays.asList(1, 2, 3), newLeaderAndIsr.isr()); + assertEquals(List.of(1, 2, 3), newLeaderAndIsr.isr()); } @Test public void testNewEpoch() { - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(3, Arrays.asList(1, 2, 3)); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(3, List.of(1, 2, 3)); assertEquals(0, leaderAndIsr.leaderEpoch()); @@ -71,7 +70,7 @@ public final class LeaderAndIsrTest { @Test public void testLeaderOpt() { - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, Arrays.asList(1, 2, 3)); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, List.of(1, 2, 3)); assertEquals(2, leaderAndIsr.leaderOpt().orElse(0)); } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ListenerInfoTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ListenerInfoTest.java index a0f9fa3b379..effde563917 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ListenerInfoTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ListenerInfoTest.java @@ -26,8 +26,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -59,7 +57,7 @@ public class ListenerInfoTest { "example2.com", 9094); - private static final List ALL = Arrays.asList( + private static final List ALL = List.of( INTERNAL, EXTERNAL, SSL, @@ -67,23 +65,23 @@ public class ListenerInfoTest { @Test public void testNullHostname() { - assertNull(ListenerInfo.create(Collections.singletonList(INTERNAL)).firstListener().host()); + assertNull(ListenerInfo.create(List.of(INTERNAL)).firstListener().host()); } @Test public void testNullHostnameGetsResolved() throws Exception { - assertNotNull(ListenerInfo.create(Collections.singletonList(INTERNAL)). + assertNotNull(ListenerInfo.create(List.of(INTERNAL)). withWildcardHostnamesResolved().firstListener().host()); } @Test public void testEmptyHostname() { - assertEquals("", ListenerInfo.create(Collections.singletonList(SSL)).firstListener().host()); + assertEquals("", ListenerInfo.create(List.of(SSL)).firstListener().host()); } @Test public void testEmptyHostnameGetsResolved() throws Exception { - assertNotEquals("", ListenerInfo.create(Collections.singletonList(SSL)). + assertNotEquals("", ListenerInfo.create(List.of(SSL)). withWildcardHostnamesResolved().firstListener().host()); } @@ -120,14 +118,14 @@ public class ListenerInfoTest { @Test public void testToControllerRegistrationRequestFailsOnNullHost() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). toControllerRegistrationRequest()); } @Test public void testToControllerRegistrationRequestFailsOnZeroPort() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). withWildcardHostnamesResolved(). toControllerRegistrationRequest()); } @@ -145,14 +143,14 @@ public class ListenerInfoTest { @Test public void testToControllerRegistrationRecordFailsOnNullHost() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). toControllerRegistrationRecord()); } @Test public void testToControllerRegistrationRecordFailsOnZeroPort() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). withWildcardHostnamesResolved(). toControllerRegistrationRecord()); } @@ -170,14 +168,14 @@ public class ListenerInfoTest { @Test public void testToBrokerRegistrationRequestFailsOnNullHost() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). toBrokerRegistrationRequest()); } @Test public void testToBrokerRegistrationRequestFailsOnZeroPort() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). withWildcardHostnamesResolved(). toBrokerRegistrationRequest()); } @@ -195,21 +193,21 @@ public class ListenerInfoTest { @Test public void testToBrokerRegistrationRecordFailsOnNullHost() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). toBrokerRegistrationRecord()); } @Test public void testToBrokerRegistrationRecordFailsOnZeroPort() { assertThrows(RuntimeException.class, - () -> ListenerInfo.create(Collections.singletonList(INTERNAL)). + () -> ListenerInfo.create(List.of(INTERNAL)). withWildcardHostnamesResolved(). toBrokerRegistrationRecord()); } @Test public void testToString() { - ListenerInfo listenerInfo = ListenerInfo.create(Arrays.asList(EXTERNAL, SASL_PLAINTEXT)); + ListenerInfo listenerInfo = ListenerInfo.create(List.of(EXTERNAL, SASL_PLAINTEXT)); assertEquals("ListenerInfo(Endpoint(listenerName='EXTERNAL', securityProtocol=SASL_SSL, host='example.com', port=9092), " + "Endpoint(listenerName='SASL_PLAINTEXT', securityProtocol=SASL_PLAINTEXT, host='example2.com', port=9094))", listenerInfo.toString()); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java index 7d15ea32ddb..585dc842522 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/PartitionRegistrationTest.java @@ -41,8 +41,6 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.stream.Stream; @@ -77,7 +75,7 @@ public class PartitionRegistrationTest { setReplicas(new int[]{1, 2, 3}).setDirectories(DirectoryId.unassignedArray(3)). setIsr(new int[]{1}).setLastKnownElr(new int[]{3}).setElr(new int[]{2}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(0).setPartitionEpoch(1).build(); assertEquals(b, a.merge(new PartitionChangeRecord(). - setLeader(3).setIsr(Collections.singletonList(3)))); + setLeader(3).setIsr(List.of(3)))); assertEquals("isr: [1, 2] -> [3], leader: 1 -> 3, leaderEpoch: 0 -> 1, partitionEpoch: 0 -> 1", b.diff(a)); assertEquals("isr: [1, 2] -> [1], elr: [] -> [2], lastKnownElr: [] -> [3], partitionEpoch: 0 -> 1", @@ -123,11 +121,11 @@ public class PartitionRegistrationTest { setControllerEpoch(-1). setLeader(1). setLeaderEpoch(123). - setIsr(Arrays.asList(1, 2)). + setIsr(List.of(1, 2)). setPartitionEpoch(456). - setReplicas(Arrays.asList(1, 2, 3)). - setAddingReplicas(Collections.emptyList()). - setRemovingReplicas(Collections.emptyList()). + setReplicas(List.of(1, 2, 3)). + setAddingReplicas(List.of()). + setRemovingReplicas(List.of()). setIsNew(true).toString(), a.toLeaderAndIsrPartitionState(new TopicPartition("foo", 1), true).toString()); assertEquals(new LeaderAndIsrRequest.PartitionState(). @@ -136,11 +134,11 @@ public class PartitionRegistrationTest { setControllerEpoch(-1). setLeader(2). setLeaderEpoch(234). - setIsr(Arrays.asList(2, 3, 4)). + setIsr(List.of(2, 3, 4)). setPartitionEpoch(567). - setReplicas(Arrays.asList(2, 3, 4)). - setAddingReplicas(Collections.emptyList()). - setRemovingReplicas(Collections.emptyList()). + setReplicas(List.of(2, 3, 4)). + setAddingReplicas(List.of()). + setRemovingReplicas(List.of()). setIsNew(false).toString(), b.toLeaderAndIsrPartitionState(new TopicPartition("bar", 0), false).toString()); } @@ -154,19 +152,19 @@ public class PartitionRegistrationTest { setDirectories(new Uuid[]{dir1, dir2, dir3}). setIsr(new int[] {1, 2, 3}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(200).build(); PartitionRegistration partition1 = partition0.merge(new PartitionChangeRecord(). - setRemovingReplicas(Collections.singletonList(3)). - setAddingReplicas(Collections.singletonList(4)). - setReplicas(Arrays.asList(1, 2, 3, 4)). - setDirectories(Arrays.asList(dir1, dir2, dir3, DirectoryId.UNASSIGNED))); + setRemovingReplicas(List.of(3)). + setAddingReplicas(List.of(4)). + setReplicas(List.of(1, 2, 3, 4)). + setDirectories(List.of(dir1, dir2, dir3, DirectoryId.UNASSIGNED))); assertEquals(new PartitionRegistration.Builder().setReplicas(new int[] {1, 2, 3, 4}). setDirectories(new Uuid[]{dir1, dir2, dir3, DirectoryId.UNASSIGNED}). setIsr(new int[] {1, 2, 3}).setRemovingReplicas(new int[] {3}).setAddingReplicas(new int[] {4}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(201).build(), partition1); PartitionRegistration partition2 = partition1.merge(new PartitionChangeRecord(). - setIsr(Arrays.asList(1, 2, 4)). - setRemovingReplicas(Collections.emptyList()). - setAddingReplicas(Collections.emptyList()). - setReplicas(Arrays.asList(1, 2, 4)). - setDirectories(Arrays.asList(dir1, dir2, DirectoryId.UNASSIGNED))); + setIsr(List.of(1, 2, 4)). + setRemovingReplicas(List.of()). + setAddingReplicas(List.of()). + setReplicas(List.of(1, 2, 4)). + setDirectories(List.of(dir1, dir2, DirectoryId.UNASSIGNED))); assertEquals(new PartitionRegistration.Builder().setReplicas(new int[] {1, 2, 4}). setDirectories(new Uuid[]{dir1, dir2, DirectoryId.UNASSIGNED}). setIsr(new int[] {1, 2, 4}).setLeader(1).setLeaderRecoveryState(LeaderRecoveryState.RECOVERED).setLeaderEpoch(100).setPartitionEpoch(202).build(), partition2); @@ -316,19 +314,19 @@ public class PartitionRegistrationTest { PartitionRecord expectRecord = new PartitionRecord(). setTopicId(topicID). setPartitionId(0). - setReplicas(Arrays.asList(0, 1, 2, 3, 4)). - setIsr(Arrays.asList(0, 1)). + setReplicas(List.of(0, 1, 2, 3, 4)). + setIsr(List.of(0, 1)). setLeader(0). setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setLeaderEpoch(0). setPartitionEpoch(0); if (metadataVersion.isElrSupported()) { expectRecord. - setEligibleLeaderReplicas(Arrays.asList(2, 3)). - setLastKnownElr(Collections.singletonList(4)); + setEligibleLeaderReplicas(List.of(2, 3)). + setLastKnownElr(List.of(4)); } if (metadataVersion.isDirectoryAssignmentSupported()) { - expectRecord.setDirectories(Arrays.asList( + expectRecord.setDirectories(List.of( DirectoryId.UNASSIGNED, Uuid.fromString("KBJBm9GVRAG9Ffe25odmmg"), DirectoryId.LOST, @@ -365,12 +363,12 @@ public class PartitionRegistrationTest { PartitionRecord expectRecord = new PartitionRecord(). setTopicId(topicID). setPartitionId(0). - setReplicas(Arrays.asList(0, 1, 2, 3, 4)). - setIsr(Arrays.asList(0, 1)). + setReplicas(List.of(0, 1, 2, 3, 4)). + setIsr(List.of(0, 1)). setLeader(0). setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setLeaderEpoch(0). - setDirectories(Arrays.asList(DirectoryId.migratingArray(5))). + setDirectories(List.of(DirectoryId.migratingArray(5))). setPartitionEpoch(0); List exceptions = new ArrayList<>(); ImageWriterOptions options = new ImageWriterOptions.Builder(MetadataVersion.IBP_4_0_IV1). @@ -451,8 +449,8 @@ public class PartitionRegistrationTest { PartitionRecord record = new PartitionRecord(). setTopicId(Uuid.fromString("ONlQ7DDzQtGESsG499UDQg")). setPartitionId(0). - setReplicas(Arrays.asList(0, 1)). - setIsr(Arrays.asList(0, 1)). + setReplicas(List.of(0, 1)). + setIsr(List.of(0, 1)). setLeader(0). setLeaderRecoveryState(LeaderRecoveryState.RECOVERED.value()). setLeaderEpoch(0). diff --git a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java index 70ec1425308..21bd04e2643 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/RecordTestUtils.java @@ -36,8 +36,6 @@ import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.Iterator; @@ -92,7 +90,7 @@ public class RecordTestUtils { Object target, ApiMessageAndVersion recordAndVersion ) { - replayAll(target, Collections.singletonList(recordAndVersion)); + replayAll(target, List.of(recordAndVersion)); } public static Optional recordAtIndexAs( @@ -357,7 +355,7 @@ public class RecordTestUtils { setIncarnationId(new Uuid(3465346L, id)). setZkMigrationReady(zkMigrationReady). setEndPoints(new RegisterControllerRecord.ControllerEndpointCollection( - Arrays.asList( + List.of( new RegisterControllerRecord.ControllerEndpoint(). setName("CONTROLLER"). setHost("localhost"). @@ -371,7 +369,7 @@ public class RecordTestUtils { ).iterator() )). setFeatures(new RegisterControllerRecord.ControllerFeatureCollection( - Collections.singletonList( + List.of( new RegisterControllerRecord.ControllerFeature(). setName(MetadataVersion.FEATURE_NAME). setMinSupportedVersion(MetadataVersion.MINIMUM_VERSION.featureLevel()). diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ReplicasTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ReplicasTest.java index 365c5eb1690..53b26680cb1 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ReplicasTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ReplicasTest.java @@ -20,9 +20,9 @@ package org.apache.kafka.metadata; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; +import java.util.List; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -34,16 +34,16 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class ReplicasTest { @Test public void testToList() { - assertEquals(Arrays.asList(1, 2, 3, 4), Replicas.toList(new int[] {1, 2, 3, 4})); - assertEquals(Collections.emptyList(), Replicas.toList(Replicas.NONE)); - assertEquals(Collections.singletonList(2), Replicas.toList(new int[] {2})); + assertEquals(List.of(1, 2, 3, 4), Replicas.toList(new int[] {1, 2, 3, 4})); + assertEquals(List.of(), Replicas.toList(Replicas.NONE)); + assertEquals(List.of(2), Replicas.toList(new int[] {2})); } @Test public void testToArray() { - assertArrayEquals(new int[] {3, 2, 1}, Replicas.toArray(Arrays.asList(3, 2, 1))); - assertArrayEquals(new int[] {}, Replicas.toArray(Collections.emptyList())); - assertArrayEquals(new int[] {2}, Replicas.toArray(Collections.singletonList(2))); + assertArrayEquals(new int[] {3, 2, 1}, Replicas.toArray(List.of(3, 2, 1))); + assertArrayEquals(new int[] {}, Replicas.toArray(List.of())); + assertArrayEquals(new int[] {2}, Replicas.toArray(List.of(2))); } @Test @@ -108,21 +108,21 @@ public class ReplicasTest { @Test public void testToSet() { - assertEquals(Collections.emptySet(), Replicas.toSet(new int[] {})); - assertEquals(new HashSet<>(Arrays.asList(3, 1, 5)), + assertEquals(Set.of(), Replicas.toSet(new int[] {})); + assertEquals(new HashSet<>(List.of(3, 1, 5)), Replicas.toSet(new int[] {1, 3, 5})); - assertEquals(new HashSet<>(Arrays.asList(1, 2, 10)), + assertEquals(new HashSet<>(List.of(1, 2, 10)), Replicas.toSet(new int[] {1, 1, 2, 10, 10})); } @Test public void testContains2() { - assertTrue(Replicas.contains(Collections.emptyList(), Replicas.NONE)); - assertFalse(Replicas.contains(Collections.emptyList(), new int[] {1})); - assertTrue(Replicas.contains(Arrays.asList(1, 2, 3), new int[] {3, 2, 1})); - assertTrue(Replicas.contains(Arrays.asList(1, 2, 3, 4), new int[] {3})); - assertTrue(Replicas.contains(Arrays.asList(1, 2, 3, 4), new int[] {3, 1})); - assertFalse(Replicas.contains(Arrays.asList(1, 2, 3, 4), new int[] {3, 1, 7})); - assertTrue(Replicas.contains(Arrays.asList(1, 2, 3, 4), new int[] {})); + assertTrue(Replicas.contains(List.of(), Replicas.NONE)); + assertFalse(Replicas.contains(List.of(), new int[] {1})); + assertTrue(Replicas.contains(List.of(1, 2, 3), new int[] {3, 2, 1})); + assertTrue(Replicas.contains(List.of(1, 2, 3, 4), new int[] {3})); + assertTrue(Replicas.contains(List.of(1, 2, 3, 4), new int[] {3, 1})); + assertFalse(Replicas.contains(List.of(1, 2, 3, 4), new int[] {3, 1, 7})); + assertTrue(Replicas.contains(List.of(1, 2, 3, 4), new int[] {})); } } diff --git a/metadata/src/test/java/org/apache/kafka/metadata/ScramCredentialDataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/ScramCredentialDataTest.java index 499b27cf31e..cdef7ead122 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/ScramCredentialDataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/ScramCredentialDataTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.server.util.MockRandom; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.List; import java.util.Random; @@ -43,7 +42,7 @@ public class ScramCredentialDataTest { return buf; } - private static final List SCRAMCREDENTIALDATA = Arrays.asList( + private static final List SCRAMCREDENTIALDATA = List.of( new ScramCredentialData( randomBuffer(random, 1024), randomBuffer(random, 1024), diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizerTest.java index ada2013e114..3fd0f96149a 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/ClusterMetadataAuthorizerTest.java @@ -39,12 +39,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; @@ -158,7 +157,7 @@ public class ClusterMetadataAuthorizerTest { } } - static final List TEST_BINDINGS = Arrays.asList( + static final List TEST_BINDINGS = List.of( new AclBinding(new ResourcePattern(TOPIC, WILDCARD_RESOURCE, LITERAL), new AccessControlEntry(WILDCARD_PRINCIPAL, WILDCARD, READ, ALLOW)), new AclBinding(new ResourcePattern(TOPIC, WILDCARD_RESOURCE, LITERAL), @@ -174,7 +173,7 @@ public class ClusterMetadataAuthorizerTest { MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer(); authorizer.setAclMutator(mutator); CompletableFuture> response = new CompletableFuture<>(); - response.complete(Arrays.asList(AclCreateResult.SUCCESS, + response.complete(List.of(AclCreateResult.SUCCESS, new AclCreateResult(new InvalidRequestException("invalid")))); mutator.setCreateAclsResponse(response); List> results = authorizer.createAcls( @@ -208,8 +207,8 @@ public class ClusterMetadataAuthorizerTest { MockClusterMetadataAuthorizer authorizer = new MockClusterMetadataAuthorizer(); authorizer.setAclMutator(mutator); CompletableFuture> response = new CompletableFuture<>(); - response.complete(Arrays.asList(new AclDeleteResult( - Collections.singleton(new AclBindingDeleteResult(TEST_BINDINGS.get(0)))), + response.complete(List.of(new AclDeleteResult( + Set.of(new AclBindingDeleteResult(TEST_BINDINGS.get(0)))), new AclDeleteResult(new InvalidRequestException("invalid")))); mutator.setDeleteAclsResponse(response); List> results = authorizer.deleteAcls( diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAclRecordIteratorTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAclRecordIteratorTest.java index 4abddc4658f..cf6bd9b5114 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAclRecordIteratorTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAclRecordIteratorTest.java @@ -22,8 +22,7 @@ import org.apache.kafka.server.common.ApiMessageAndVersion; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; import java.util.NoSuchElementException; import static org.apache.kafka.metadata.authorizer.StandardAclWithIdTest.TEST_ACLS; @@ -40,16 +39,16 @@ public class StandardAclRecordIteratorTest { StandardAclRecordIterator iterator = new StandardAclRecordIterator(TEST_ACLS.iterator(), 2); assertTrue(iterator.hasNext()); - assertEquals(Arrays.asList( + assertEquals(List.of( new ApiMessageAndVersion(TEST_ACLS.get(0).toRecord(), (short) 0), new ApiMessageAndVersion(TEST_ACLS.get(1).toRecord(), (short) 0)), iterator.next()); - assertEquals(Arrays.asList( + assertEquals(List.of( new ApiMessageAndVersion(TEST_ACLS.get(2).toRecord(), (short) 0), new ApiMessageAndVersion(TEST_ACLS.get(3).toRecord(), (short) 0)), iterator.next()); assertTrue(iterator.hasNext()); - assertEquals(Collections.singletonList( + assertEquals(List.of( new ApiMessageAndVersion(TEST_ACLS.get(4).toRecord(), (short) 0)), iterator.next()); assertFalse(iterator.hasNext()); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerPropertyTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerPropertyTest.java index 5c532c48737..2090fa710a6 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerPropertyTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerPropertyTest.java @@ -44,7 +44,6 @@ import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Random; @@ -162,7 +161,7 @@ public class StandardAuthorizerPropertyTest { private StandardAuthorizer buildAuthorizer() { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.start(new StandardAuthorizerTest.AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT))); + authorizer.start(new StandardAuthorizerTest.AuthorizerTestServerInfo(List.of(PLAINTEXT))); authorizer.completeInitialLoad(); return authorizer; } @@ -176,7 +175,7 @@ public class StandardAuthorizerPropertyTest { Action action = new Action(operation, pattern, 1, false, false); List results = authorizer.authorize( newRequestContext(), - Collections.singletonList(action) + List.of(action) ); assertEquals(1, results.size()); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java index bfa40c7c2e0..834aa65cb69 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/authorizer/StandardAuthorizerTest.java @@ -48,19 +48,16 @@ import org.slf4j.LoggerFactory; import java.net.InetAddress; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletionStage; import java.util.concurrent.atomic.AtomicLong; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.apache.kafka.common.acl.AclOperation.ALL; import static org.apache.kafka.common.acl.AclOperation.ALTER; import static org.apache.kafka.common.acl.AclOperation.ALTER_CONFIGS; @@ -147,25 +144,25 @@ public class StandardAuthorizerTest { @Test public void testGetConfiguredSuperUsers() { - assertEquals(Collections.emptySet(), - getConfiguredSuperUsers(Collections.emptyMap())); - assertEquals(Collections.emptySet(), - getConfiguredSuperUsers(Collections.singletonMap(SUPER_USERS_CONFIG, " "))); - assertEquals(new HashSet<>(asList("User:bob", "User:alice")), - getConfiguredSuperUsers(Collections.singletonMap(SUPER_USERS_CONFIG, "User:bob;User:alice "))); - assertEquals(new HashSet<>(asList("User:bob", "User:alice")), - getConfiguredSuperUsers(Collections.singletonMap(SUPER_USERS_CONFIG, "; User:bob ; User:alice "))); + assertEquals(Set.of(), + getConfiguredSuperUsers(Map.of())); + assertEquals(Set.of(), + getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, " "))); + assertEquals(new HashSet<>(List.of("User:bob", "User:alice")), + getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "User:bob;User:alice "))); + assertEquals(new HashSet<>(List.of("User:bob", "User:alice")), + getConfiguredSuperUsers(Map.of(SUPER_USERS_CONFIG, "; User:bob ; User:alice "))); assertEquals("expected a string in format principalType:principalName but got bob", assertThrows(IllegalArgumentException.class, () -> getConfiguredSuperUsers( - Collections.singletonMap(SUPER_USERS_CONFIG, "bob;:alice"))).getMessage()); + Map.of(SUPER_USERS_CONFIG, "bob;:alice"))).getMessage()); } @Test public void testGetDefaultResult() { - assertEquals(DENIED, getDefaultResult(Collections.emptyMap())); - assertEquals(ALLOWED, getDefaultResult(Collections.singletonMap( + assertEquals(DENIED, getDefaultResult(Map.of())); + assertEquals(ALLOWED, getDefaultResult(Map.of( ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true"))); - assertEquals(DENIED, getDefaultResult(Collections.singletonMap( + assertEquals(DENIED, getDefaultResult(Map.of( ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "false"))); } @@ -176,26 +173,26 @@ public class StandardAuthorizerTest { configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true"); authorizer.configure(configs); - authorizer.start(new AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT))); + authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT))); authorizer.completeInitialLoad(); - List acls = singletonList( + List acls = List.of( withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW)) ); acls.forEach(acl -> authorizer.addAcl(acl.id(), acl.acl())); - assertEquals(singletonList(DENIED), + assertEquals(List.of(DENIED), authorizer.authorize( new MockAuthorizableRequestContext.Builder() .setPrincipal(new KafkaPrincipal(USER_TYPE, "Bob")) .build(), - singletonList(newAction(READ, TOPIC, "topic1")) + List.of(newAction(READ, TOPIC, "topic1")) )); - assertEquals(singletonList(ALLOWED), + assertEquals(List.of(ALLOWED), authorizer.authorize( new MockAuthorizableRequestContext.Builder() .setPrincipal(new KafkaPrincipal(USER_TYPE, "Bob")) .build(), - singletonList(newAction(READ, TOPIC, "topic2")) + List.of(newAction(READ, TOPIC, "topic2")) )); } @@ -206,26 +203,26 @@ public class StandardAuthorizerTest { configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "false"); authorizer.configure(configs); - authorizer.start(new AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT))); + authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT))); authorizer.completeInitialLoad(); - List acls = singletonList( + List acls = List.of( withId(new StandardAcl(TOPIC, "topic1", LITERAL, "User:Alice", WILDCARD, READ, ALLOW)) ); acls.forEach(acl -> authorizer.addAcl(acl.id(), acl.acl())); - assertEquals(singletonList(DENIED), + assertEquals(List.of(DENIED), authorizer.authorize( new MockAuthorizableRequestContext.Builder() .setPrincipal(new KafkaPrincipal(USER_TYPE, "Bob")) .build(), - singletonList(newAction(READ, TOPIC, "topic1")) + List.of(newAction(READ, TOPIC, "topic1")) )); - assertEquals(singletonList(DENIED), + assertEquals(List.of(DENIED), authorizer.authorize( new MockAuthorizableRequestContext.Builder() .setPrincipal(new KafkaPrincipal(USER_TYPE, "Bob")) .build(), - singletonList(newAction(READ, TOPIC, "topic2")) + List.of(newAction(READ, TOPIC, "topic2")) )); } @@ -236,7 +233,7 @@ public class StandardAuthorizerTest { configs.put(SUPER_USERS_CONFIG, "User:alice;User:chris"); configs.put(ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true"); authorizer.configure(configs); - assertEquals(new HashSet<>(asList("User:alice", "User:chris")), authorizer.superUsers()); + assertEquals(new HashSet<>(List.of("User:alice", "User:chris")), authorizer.superUsers()); assertEquals(ALLOWED, authorizer.defaultResult()); } @@ -249,8 +246,8 @@ public class StandardAuthorizerTest { static StandardAuthorizer createAndInitializeStandardAuthorizer() { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.configure(Collections.singletonMap(SUPER_USERS_CONFIG, "User:superman")); - authorizer.start(new AuthorizerTestServerInfo(Collections.singletonList(PLAINTEXT))); + authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); + authorizer.start(new AuthorizerTestServerInfo(List.of(PLAINTEXT))); authorizer.completeInitialLoad(); return authorizer; } @@ -275,7 +272,7 @@ public class StandardAuthorizerTest { @Test public void testFindResultImplication() throws Exception { // These permissions all imply DESCRIBE. - for (AclOperation op : asList(DESCRIBE, READ, WRITE, DELETE, ALTER)) { + for (AclOperation op : List.of(DESCRIBE, READ, WRITE, DELETE, ALTER)) { assertEquals(ALLOWED, findResult(newAction(DESCRIBE, TOPIC, "foo_bar"), new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), @@ -287,7 +284,7 @@ public class StandardAuthorizerTest { setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), newFooAcl(CREATE, ALLOW))); // Deny ACLs don't do "implication". - for (AclOperation op : asList(READ, WRITE, DELETE, ALTER)) { + for (AclOperation op : List.of(READ, WRITE, DELETE, ALTER)) { assertNull(findResult(newAction(DESCRIBE, TOPIC, "foo_bar"), new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), @@ -299,7 +296,7 @@ public class StandardAuthorizerTest { setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), newFooAcl(DESCRIBE, DENY))); // These permissions all imply DESCRIBE_CONFIGS. - for (AclOperation op : asList(DESCRIBE_CONFIGS, ALTER_CONFIGS)) { + for (AclOperation op : List.of(DESCRIBE_CONFIGS, ALTER_CONFIGS)) { assertEquals(ALLOWED, findResult(newAction(DESCRIBE_CONFIGS, TOPIC, "foo_bar"), new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), @@ -359,10 +356,10 @@ public class StandardAuthorizerTest { @Test public void testListAcls() { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List fooAcls = asList( + List fooAcls = List.of( withId(newFooAcl(READ, ALLOW)), withId(newFooAcl(WRITE, ALLOW))); - List barAcls = asList( + List barAcls = List.of( withId(newBarAcl(DESCRIBE_CONFIGS, DENY)), withId(newBarAcl(ALTER_CONFIGS, DENY))); fooAcls.forEach(a -> authorizer.addAcl(a.id(), a.acl())); @@ -380,28 +377,28 @@ public class StandardAuthorizerTest { @Test public void testSimpleAuthorizations() throws Exception { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List fooAcls = asList( + List fooAcls = List.of( withId(newFooAcl(READ, ALLOW)), withId(newFooAcl(WRITE, ALLOW))); - List barAcls = asList( + List barAcls = List.of( withId(newBarAcl(DESCRIBE_CONFIGS, ALLOW)), withId(newBarAcl(ALTER_CONFIGS, ALLOW))); fooAcls.forEach(a -> authorizer.addAcl(a.id(), a.acl())); barAcls.forEach(a -> authorizer.addAcl(a.id(), a.acl())); - assertEquals(singletonList(ALLOWED), + assertEquals(List.of(ALLOWED), authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), - singletonList(newAction(READ, TOPIC, "foo_")))); - assertEquals(singletonList(ALLOWED), + List.of(newAction(READ, TOPIC, "foo_")))); + assertEquals(List.of(ALLOWED), authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "fred")).build(), - singletonList(newAction(ALTER_CONFIGS, GROUP, "bar")))); + List.of(newAction(ALTER_CONFIGS, GROUP, "bar")))); } @Test public void testDenyPrecedenceWithOperationAll() throws Exception { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List acls = Arrays.asList( + List acls = List.of( new StandardAcl(TOPIC, "foo", LITERAL, "User:alice", "*", ALL, DENY), new StandardAcl(TOPIC, "foo", PREFIXED, "User:alice", "*", READ, ALLOW), new StandardAcl(TOPIC, "foo", LITERAL, "User:*", "*", ALL, DENY), @@ -411,16 +408,16 @@ public class StandardAuthorizerTest { StandardAclWithId aclWithId = withId(acl); authorizer.addAcl(aclWithId.id(), aclWithId.acl()); }); - assertEquals(Arrays.asList(DENIED, DENIED, DENIED, ALLOWED), authorizer.authorize( + assertEquals(List.of(DENIED, DENIED, DENIED, ALLOWED), authorizer.authorize( newRequestContext("alice"), - Arrays.asList( + List.of( newAction(WRITE, TOPIC, "foo"), newAction(READ, TOPIC, "foo"), newAction(DESCRIBE, TOPIC, "foo"), newAction(READ, TOPIC, "foobar")))); - assertEquals(Arrays.asList(DENIED, DENIED, DENIED, ALLOWED, DENIED), authorizer.authorize( + assertEquals(List.of(DENIED, DENIED, DENIED, ALLOWED, DENIED), authorizer.authorize( newRequestContext("bob"), - Arrays.asList( + List.of( newAction(DESCRIBE, TOPIC, "foo"), newAction(READ, TOPIC, "foo"), newAction(WRITE, TOPIC, "foo"), @@ -431,7 +428,7 @@ public class StandardAuthorizerTest { @Test public void testTopicAclWithOperationAll() throws Exception { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List acls = Arrays.asList( + List acls = List.of( new StandardAcl(TOPIC, "foo", LITERAL, "User:*", "*", ALL, ALLOW), new StandardAcl(TOPIC, "bar", PREFIXED, "User:alice", "*", ALL, ALLOW), new StandardAcl(TOPIC, "baz", LITERAL, "User:bob", "*", ALL, ALLOW) @@ -440,23 +437,23 @@ public class StandardAuthorizerTest { StandardAclWithId aclWithId = withId(acl); authorizer.addAcl(aclWithId.id(), aclWithId.acl()); }); - assertEquals(Arrays.asList(ALLOWED, ALLOWED, DENIED), authorizer.authorize( + assertEquals(List.of(ALLOWED, ALLOWED, DENIED), authorizer.authorize( newRequestContext("alice"), - Arrays.asList( + List.of( newAction(WRITE, TOPIC, "foo"), newAction(DESCRIBE_CONFIGS, TOPIC, "bar"), newAction(DESCRIBE, TOPIC, "baz")))); - assertEquals(Arrays.asList(ALLOWED, DENIED, ALLOWED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, ALLOWED), authorizer.authorize( newRequestContext("bob"), - Arrays.asList( + List.of( newAction(WRITE, TOPIC, "foo"), newAction(READ, TOPIC, "bar"), newAction(DESCRIBE, TOPIC, "baz")))); - assertEquals(Arrays.asList(ALLOWED, DENIED, DENIED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, DENIED), authorizer.authorize( newRequestContext("malory"), - Arrays.asList( + List.of( newAction(DESCRIBE, TOPIC, "foo"), newAction(WRITE, TOPIC, "bar"), newAction(READ, TOPIC, "baz")))); @@ -474,7 +471,7 @@ public class StandardAuthorizerTest { InetAddress host2 = InetAddress.getByName("192.168.1.2"); StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List acls = Arrays.asList( + List acls = List.of( new StandardAcl(TOPIC, "foo", LITERAL, "User:alice", host1.getHostAddress(), READ, DENY), new StandardAcl(TOPIC, "foo", LITERAL, "User:alice", "*", READ, ALLOW), new StandardAcl(TOPIC, "bar", LITERAL, "User:bob", host2.getHostAddress(), READ, ALLOW), @@ -486,28 +483,28 @@ public class StandardAuthorizerTest { authorizer.addAcl(aclWithId.id(), aclWithId.acl()); }); - List actions = Arrays.asList( + List actions = List.of( newAction(READ, TOPIC, "foo"), newAction(READ, TOPIC, "bar"), newAction(DESCRIBE, TOPIC, "bar") ); - assertEquals(Arrays.asList(ALLOWED, DENIED, ALLOWED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, ALLOWED), authorizer.authorize( newRequestContext("alice", InetAddress.getLocalHost()), actions)); - assertEquals(Arrays.asList(DENIED, DENIED, DENIED), authorizer.authorize( + assertEquals(List.of(DENIED, DENIED, DENIED), authorizer.authorize( newRequestContext("alice", host1), actions)); - assertEquals(Arrays.asList(ALLOWED, DENIED, DENIED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, DENIED), authorizer.authorize( newRequestContext("alice", host2), actions)); - assertEquals(Arrays.asList(DENIED, DENIED, ALLOWED), authorizer.authorize( + assertEquals(List.of(DENIED, DENIED, ALLOWED), authorizer.authorize( newRequestContext("bob", InetAddress.getLocalHost()), actions)); - assertEquals(Arrays.asList(DENIED, DENIED, DENIED), authorizer.authorize( + assertEquals(List.of(DENIED, DENIED, DENIED), authorizer.authorize( newRequestContext("bob", host1), actions)); - assertEquals(Arrays.asList(DENIED, ALLOWED, ALLOWED), authorizer.authorize( + assertEquals(List.of(DENIED, ALLOWED, ALLOWED), authorizer.authorize( newRequestContext("bob", host2), actions)); } @@ -519,7 +516,7 @@ public class StandardAuthorizerTest { } private static void addManyAcls(StandardAuthorizer authorizer) { - List acls = Arrays.asList( + List acls = List.of( new StandardAcl(TOPIC, "green2", LITERAL, "User:*", "*", READ, ALLOW), new StandardAcl(TOPIC, "green", PREFIXED, "User:bob", "*", READ, ALLOW), new StandardAcl(TOPIC, "betamax4", LITERAL, "User:bob", "*", READ, ALLOW), @@ -540,15 +537,15 @@ public class StandardAuthorizerTest { public void testAuthorizationWithManyAcls() throws Exception { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); addManyAcls(authorizer); - assertEquals(Arrays.asList(ALLOWED, DENIED), + assertEquals(List.of(ALLOWED, DENIED), authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), - Arrays.asList(newAction(READ, TOPIC, "green1"), + List.of(newAction(READ, TOPIC, "green1"), newAction(WRITE, GROUP, "wheel")))); - assertEquals(Arrays.asList(DENIED, ALLOWED, DENIED), + assertEquals(List.of(DENIED, ALLOWED, DENIED), authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), - Arrays.asList(newAction(READ, TOPIC, "alpha"), + List.of(newAction(READ, TOPIC, "alpha"), newAction(WRITE, GROUP, "arbitrary"), newAction(READ, TOPIC, "ala")))); } @@ -579,7 +576,7 @@ public class StandardAuthorizerTest { .setClientAddress(InetAddress.getByName("127.0.0.1")) .build(); - assertEquals(singletonList(DENIED), authorizer.authorize(requestContext, singletonList(action))); + assertEquals(List.of(DENIED), authorizer.authorize(requestContext, List.of(action))); String expectedAuditLog = "Principal = User:bob is Denied operation = READ " + "from host = 127.0.0.1 on resource = Topic:LITERAL:alpha for request = Fetch " + @@ -621,7 +618,7 @@ public class StandardAuthorizerTest { .setClientAddress(InetAddress.getByName("127.0.0.1")) .build(); - assertEquals(singletonList(ALLOWED), authorizer.authorize(requestContext, singletonList(action))); + assertEquals(List.of(ALLOWED), authorizer.authorize(requestContext, List.of(action))); String expectedAuditLog = "Principal = User:bob is Allowed operation = READ " + "from host = 127.0.0.1 on resource = Topic:LITERAL:green1 for request = Fetch " + @@ -644,10 +641,10 @@ public class StandardAuthorizerTest { @Test public void testStartWithEarlyStartListeners() { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.configure(Collections.singletonMap(SUPER_USERS_CONFIG, "User:superman")); + authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); Map> futures2 = authorizer. - start(new AuthorizerTestServerInfo(Arrays.asList(PLAINTEXT, CONTROLLER))); - assertEquals(new HashSet<>(Arrays.asList(PLAINTEXT, CONTROLLER)), futures2.keySet()); + start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER))); + assertEquals(new HashSet<>(List.of(PLAINTEXT, CONTROLLER)), futures2.keySet()); assertFalse(futures2.get(PLAINTEXT).toCompletableFuture().isDone()); assertTrue(futures2.get(CONTROLLER).toCompletableFuture().isDone()); } @@ -661,26 +658,26 @@ public class StandardAuthorizerTest { @Test public void testAuthorizationPriorToCompleteInitialLoad() throws Exception { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.configure(Collections.singletonMap(SUPER_USERS_CONFIG, "User:superman")); + authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); assertThrows(AuthorizerNotReadyException.class, () -> authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "bob")).build(), - Arrays.asList(newAction(READ, TOPIC, "green1"), + List.of(newAction(READ, TOPIC, "green1"), newAction(READ, TOPIC, "green2")))); - assertEquals(Arrays.asList(ALLOWED, ALLOWED), + assertEquals(List.of(ALLOWED, ALLOWED), authorizer.authorize(new MockAuthorizableRequestContext.Builder(). setPrincipal(new KafkaPrincipal(USER_TYPE, "superman")).build(), - Arrays.asList(newAction(READ, TOPIC, "green1"), + List.of(newAction(READ, TOPIC, "green1"), newAction(WRITE, GROUP, "wheel")))); } @Test public void testCompleteInitialLoad() { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.configure(Collections.singletonMap(SUPER_USERS_CONFIG, "User:superman")); + authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); Map> futures = authorizer. - start(new AuthorizerTestServerInfo(Collections.singleton(PLAINTEXT))); - assertEquals(Collections.singleton(PLAINTEXT), futures.keySet()); + start(new AuthorizerTestServerInfo(Set.of(PLAINTEXT))); + assertEquals(Set.of(PLAINTEXT), futures.keySet()); assertFalse(futures.get(PLAINTEXT).toCompletableFuture().isDone()); authorizer.completeInitialLoad(); assertTrue(futures.get(PLAINTEXT).toCompletableFuture().isDone()); @@ -690,10 +687,10 @@ public class StandardAuthorizerTest { @Test public void testCompleteInitialLoadWithException() { StandardAuthorizer authorizer = new StandardAuthorizer(); - authorizer.configure(Collections.singletonMap(SUPER_USERS_CONFIG, "User:superman")); + authorizer.configure(Map.of(SUPER_USERS_CONFIG, "User:superman")); Map> futures = authorizer. - start(new AuthorizerTestServerInfo(Arrays.asList(PLAINTEXT, CONTROLLER))); - assertEquals(new HashSet<>(Arrays.asList(PLAINTEXT, CONTROLLER)), futures.keySet()); + start(new AuthorizerTestServerInfo(List.of(PLAINTEXT, CONTROLLER))); + assertEquals(new HashSet<>(List.of(PLAINTEXT, CONTROLLER)), futures.keySet()); assertFalse(futures.get(PLAINTEXT).toCompletableFuture().isDone()); assertTrue(futures.get(CONTROLLER).toCompletableFuture().isDone()); authorizer.completeInitialLoad(new TimeoutException("timed out")); @@ -706,7 +703,7 @@ public class StandardAuthorizerTest { @Test public void testPrefixAcls() throws Exception { StandardAuthorizer authorizer = createAndInitializeStandardAuthorizer(); - List acls = Arrays.asList( + List acls = List.of( new StandardAcl(TOPIC, "fooa", PREFIXED, "User:alice", "*", ALL, ALLOW), new StandardAcl(TOPIC, "foobar", LITERAL, "User:bob", "*", ALL, ALLOW), new StandardAcl(TOPIC, "f", PREFIXED, "User:bob", "*", ALL, ALLOW) @@ -715,16 +712,16 @@ public class StandardAuthorizerTest { StandardAclWithId aclWithId = withId(acl); authorizer.addAcl(aclWithId.id(), aclWithId.acl()); }); - assertEquals(Arrays.asList(ALLOWED, DENIED, ALLOWED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, ALLOWED), authorizer.authorize( newRequestContext("bob"), - Arrays.asList( + List.of( newAction(WRITE, TOPIC, "foobarr"), newAction(READ, TOPIC, "goobar"), newAction(READ, TOPIC, "fooa")))); - assertEquals(Arrays.asList(ALLOWED, DENIED, DENIED), authorizer.authorize( + assertEquals(List.of(ALLOWED, DENIED, DENIED), authorizer.authorize( newRequestContext("alice"), - Arrays.asList( + List.of( newAction(DESCRIBE, TOPIC, "fooa"), newAction(WRITE, TOPIC, "bar"), newAction(READ, TOPIC, "baz")))); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java index edee755eee3..5d19c1068c6 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/bootstrap/BootstrapMetadataTest.java @@ -26,11 +26,8 @@ import org.apache.kafka.server.common.MetadataVersionTestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; import java.util.List; -import static java.util.Collections.emptyList; import static org.apache.kafka.server.common.MetadataVersion.FEATURE_NAME; import static org.apache.kafka.server.common.MetadataVersion.IBP_3_3_IV3; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -50,7 +47,7 @@ public class BootstrapMetadataTest { @Test public void testFromVersion() { - assertEquals(new BootstrapMetadata(Collections.singletonList( + assertEquals(new BootstrapMetadata(List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). setFeatureLevel((short) 7), (short) 0)), @@ -68,7 +65,7 @@ public class BootstrapMetadataTest { public void testFromRecordsListWithoutMetadataVersion() { assertEquals("No FeatureLevelRecord for metadata.version was found in the bootstrap " + "metadata from quux", assertThrows(RuntimeException.class, - () -> BootstrapMetadata.fromRecords(emptyList(), "quux")).getMessage()); + () -> BootstrapMetadata.fromRecords(List.of(), "quux")).getMessage()); } private static final ApiMessageAndVersion MV_10 = @@ -93,38 +90,38 @@ public class BootstrapMetadataTest { @Test public void testCopyWithNewFeatureRecord() { - assertEquals(BootstrapMetadata.fromRecords(Arrays.asList(MV_10, FOO_1), "src"), - BootstrapMetadata.fromRecords(Arrays.asList(MV_10), "src"). + assertEquals(BootstrapMetadata.fromRecords(List.of(MV_10, FOO_1), "src"), + BootstrapMetadata.fromRecords(List.of(MV_10), "src"). copyWithFeatureRecord("foo", (short) 1)); } @Test public void testFeatureLevelForMetadataVersion() { assertEquals((short) 11, BootstrapMetadata. - fromRecords(Arrays.asList(MV_10, MV_11), "src"). + fromRecords(List.of(MV_10, MV_11), "src"). featureLevel(FEATURE_NAME)); } @Test public void testCopyWithModifiedFeatureRecord() { - assertEquals(BootstrapMetadata.fromRecords(Arrays.asList(MV_10, FOO_2), "src"), - BootstrapMetadata.fromRecords(Arrays.asList(MV_10, FOO_1), "src"). + assertEquals(BootstrapMetadata.fromRecords(List.of(MV_10, FOO_2), "src"), + BootstrapMetadata.fromRecords(List.of(MV_10, FOO_1), "src"). copyWithFeatureRecord("foo", (short) 2)); } @Test public void testFeatureLevelForFeatureThatIsNotSet() { assertEquals((short) 0, BootstrapMetadata. - fromRecords(Arrays.asList(MV_10), "src").featureLevel("foo")); + fromRecords(List.of(MV_10), "src").featureLevel("foo")); } @Test public void testFeatureLevelForFeature() { assertEquals((short) 2, BootstrapMetadata. - fromRecords(Arrays.asList(MV_10, FOO_2), "src").featureLevel("foo")); + fromRecords(List.of(MV_10, FOO_2), "src").featureLevel("foo")); } - static final List RECORDS_WITH_OLD_METADATA_VERSION = Collections.singletonList( + static final List RECORDS_WITH_OLD_METADATA_VERSION = List.of( new ApiMessageAndVersion(new FeatureLevelRecord(). setName(FEATURE_NAME). setFeatureLevel(MetadataVersionTestUtils.IBP_3_0_IV1_FEATURE_LEVEL), (short) 0)); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java index 2f0fdee512c..d3fd1c24cb4 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/PartitionAssignmentTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.common.Uuid; import org.junit.jupiter.api.Test; -import java.util.Arrays; import java.util.List; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -35,18 +34,18 @@ public class PartitionAssignmentTest { @Test public void testPartitionAssignmentReplicas() { - List replicas = Arrays.asList(0, 1, 2); + List replicas = List.of(0, 1, 2); assertEquals(replicas, partitionAssignment(replicas).replicas()); } @Test public void testConsistentEqualsAndHashCode() { - List partitionAssignments = Arrays.asList( + List partitionAssignments = List.of( partitionAssignment( - Arrays.asList(0, 1, 2) + List.of(0, 1, 2) ), partitionAssignment( - Arrays.asList(1, 2, 0) + List.of(1, 2, 0) ) ); @@ -66,8 +65,8 @@ public class PartitionAssignmentTest { @Test public void testToString() { - List replicas = Arrays.asList(0, 1, 2); - List directories = Arrays.asList( + List replicas = List.of(0, 1, 2); + List directories = List.of( Uuid.fromString("65WMNfybQpCDVulYOxMCTw"), Uuid.fromString("VkZ5AkuESPGkMc2OxpKUjw"), Uuid.fromString("wFtTi4FxTlOhhHytfxv7fQ") diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java index 25a70efae2a..55d992d28b6 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/StripedReplicaPlacerTest.java @@ -27,8 +27,6 @@ import org.apache.kafka.server.util.MockRandom; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -71,7 +69,7 @@ public class StripedReplicaPlacerTest { @Test public void testAvoidFencedReplicaIfPossibleOnSingleRack() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(3, Optional.empty(), false), new UsableBroker(1, Optional.empty(), true), new UsableBroker(0, Optional.empty(), false), @@ -79,14 +77,14 @@ public class StripedReplicaPlacerTest { new UsableBroker(2, Optional.empty(), false)).iterator()); assertEquals(5, rackList.numTotalBrokers()); assertEquals(4, rackList.numUnfencedBrokers()); - assertEquals(Collections.singletonList(Optional.empty()), rackList.rackNames()); + assertEquals(List.of(Optional.empty()), rackList.rackNames()); assertThrows(InvalidReplicationFactorException.class, () -> rackList.place(0)); assertThrows(InvalidReplicationFactorException.class, () -> rackList.place(-1)); - assertEquals(Arrays.asList(3, 4, 0, 2), rackList.place(4)); - assertEquals(Arrays.asList(4, 0, 2, 3), rackList.place(4)); - assertEquals(Arrays.asList(0, 2, 3, 4), rackList.place(4)); - assertEquals(Arrays.asList(2, 3, 4, 0), rackList.place(4)); - assertEquals(Arrays.asList(0, 4, 3, 2), rackList.place(4)); + assertEquals(List.of(3, 4, 0, 2), rackList.place(4)); + assertEquals(List.of(4, 0, 2, 3), rackList.place(4)); + assertEquals(List.of(0, 2, 3, 4), rackList.place(4)); + assertEquals(List.of(2, 3, 4, 0), rackList.place(4)); + assertEquals(List.of(0, 4, 3, 2), rackList.place(4)); } private TopicAssignment place( @@ -120,10 +118,10 @@ public class StripedReplicaPlacerTest { public void testMultiPartitionTopicPlacementOnSingleUnfencedBroker() { MockRandom random = new MockRandom(); StripedReplicaPlacer placer = new StripedReplicaPlacer(random); - assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Collections.singletonList(0)), - partitionAssignment(Collections.singletonList(0)), - partitionAssignment(Collections.singletonList(0)))), - place(placer, 0, 3, (short) 1, Arrays.asList( + assertEquals(new TopicAssignment(List.of(partitionAssignment(List.of(0)), + partitionAssignment(List.of(0)), + partitionAssignment(List.of(0)))), + place(placer, 0, 3, (short) 1, List.of( new UsableBroker(0, Optional.empty(), false), new UsableBroker(1, Optional.empty(), true)))); } @@ -134,23 +132,23 @@ public class StripedReplicaPlacerTest { @Test public void testPlacementOnFencedReplicaOnSingleRack() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(3, Optional.empty(), false), new UsableBroker(1, Optional.empty(), true), new UsableBroker(2, Optional.empty(), false)).iterator()); assertEquals(3, rackList.numTotalBrokers()); assertEquals(2, rackList.numUnfencedBrokers()); - assertEquals(Collections.singletonList(Optional.empty()), rackList.rackNames()); - assertEquals(Arrays.asList(3, 2, 1), rackList.place(3)); - assertEquals(Arrays.asList(2, 3, 1), rackList.place(3)); - assertEquals(Arrays.asList(3, 2, 1), rackList.place(3)); - assertEquals(Arrays.asList(2, 3, 1), rackList.place(3)); + assertEquals(List.of(Optional.empty()), rackList.rackNames()); + assertEquals(List.of(3, 2, 1), rackList.place(3)); + assertEquals(List.of(2, 3, 1), rackList.place(3)); + assertEquals(List.of(3, 2, 1), rackList.place(3)); + assertEquals(List.of(2, 3, 1), rackList.place(3)); } @Test public void testRackListWithMultipleRacks() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false), new UsableBroker(30, Optional.of("3"), false), @@ -159,16 +157,16 @@ public class StripedReplicaPlacerTest { new UsableBroker(20, Optional.of("2"), true)).iterator()); assertEquals(6, rackList.numTotalBrokers()); assertEquals(5, rackList.numUnfencedBrokers()); - assertEquals(Arrays.asList(Optional.of("1"), Optional.of("2"), Optional.of("3")), rackList.rackNames()); - assertEquals(Arrays.asList(11, 21, 31, 10), rackList.place(4)); - assertEquals(Arrays.asList(21, 30, 10, 20), rackList.place(4)); - assertEquals(Arrays.asList(31, 11, 21, 30), rackList.place(4)); + assertEquals(List.of(Optional.of("1"), Optional.of("2"), Optional.of("3")), rackList.rackNames()); + assertEquals(List.of(11, 21, 31, 10), rackList.place(4)); + assertEquals(List.of(21, 30, 10, 20), rackList.place(4)); + assertEquals(List.of(31, 11, 21, 30), rackList.place(4)); } @Test public void testRackListWithInvalidRacks() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false), new UsableBroker(30, Optional.of("3"), true), @@ -179,13 +177,13 @@ public class StripedReplicaPlacerTest { new UsableBroker(40, Optional.of("4"), true)).iterator()); assertEquals(8, rackList.numTotalBrokers()); assertEquals(3, rackList.numUnfencedBrokers()); - assertEquals(Arrays.asList(Optional.of("1"), + assertEquals(List.of(Optional.of("1"), Optional.of("2"), Optional.of("3"), Optional.of("4")), rackList.rackNames()); - assertEquals(Arrays.asList(41, 11, 21, 30), rackList.place(4)); - assertEquals(Arrays.asList(10, 20, 31, 41), rackList.place(4)); - assertEquals(Arrays.asList(41, 21, 30, 11), rackList.place(4)); + assertEquals(List.of(41, 11, 21, 30), rackList.place(4)); + assertEquals(List.of(10, 20, 31, 41), rackList.place(4)); + assertEquals(List.of(41, 21, 30, 11), rackList.place(4)); } @Test @@ -194,7 +192,7 @@ public class StripedReplicaPlacerTest { StripedReplicaPlacer placer = new StripedReplicaPlacer(random); assertEquals("All brokers are currently fenced.", assertThrows(InvalidReplicationFactorException.class, - () -> place(placer, 0, 1, (short) 1, Arrays.asList( + () -> place(placer, 0, 1, (short) 1, List.of( new UsableBroker(11, Optional.of("1"), true), new UsableBroker(10, Optional.of("1"), true)))).getMessage()); } @@ -206,7 +204,7 @@ public class StripedReplicaPlacerTest { assertEquals("The target replication factor of 3 cannot be reached because only " + "2 broker(s) are registered.", assertThrows(InvalidReplicationFactorException.class, - () -> place(placer, 0, 1, (short) 3, Arrays.asList( + () -> place(placer, 0, 1, (short) 3, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false)))).getMessage()); } @@ -217,7 +215,7 @@ public class StripedReplicaPlacerTest { StripedReplicaPlacer placer = new StripedReplicaPlacer(random); assertEquals("Invalid replication factor 0: the replication factor must be positive.", assertThrows(InvalidReplicationFactorException.class, - () -> place(placer, 0, 1, (short) 0, Arrays.asList( + () -> place(placer, 0, 1, (short) 0, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false)))).getMessage()); } @@ -226,12 +224,12 @@ public class StripedReplicaPlacerTest { public void testSuccessfulPlacement() { MockRandom random = new MockRandom(); StripedReplicaPlacer placer = new StripedReplicaPlacer(random); - assertEquals(new TopicAssignment(Arrays.asList(partitionAssignment(Arrays.asList(2, 3, 0)), - partitionAssignment(Arrays.asList(3, 0, 1)), - partitionAssignment(Arrays.asList(0, 1, 2)), - partitionAssignment(Arrays.asList(1, 2, 3)), - partitionAssignment(Arrays.asList(1, 0, 2)))), - place(placer, 0, 5, (short) 3, Arrays.asList( + assertEquals(new TopicAssignment(List.of(partitionAssignment(List.of(2, 3, 0)), + partitionAssignment(List.of(3, 0, 1)), + partitionAssignment(List.of(0, 1, 2)), + partitionAssignment(List.of(1, 2, 3)), + partitionAssignment(List.of(1, 0, 2)))), + place(placer, 0, 5, (short) 3, List.of( new UsableBroker(0, Optional.empty(), false), new UsableBroker(3, Optional.empty(), false), new UsableBroker(2, Optional.empty(), false), @@ -242,7 +240,7 @@ public class StripedReplicaPlacerTest { public void testEvenDistribution() { MockRandom random = new MockRandom(); StripedReplicaPlacer placer = new StripedReplicaPlacer(random); - TopicAssignment topicAssignment = place(placer, 0, 200, (short) 2, Arrays.asList( + TopicAssignment topicAssignment = place(placer, 0, 200, (short) 2, List.of( new UsableBroker(0, Optional.empty(), false), new UsableBroker(1, Optional.empty(), false), new UsableBroker(2, Optional.empty(), false), @@ -251,31 +249,31 @@ public class StripedReplicaPlacerTest { for (PartitionAssignment partitionAssignment : topicAssignment.assignments()) { counts.put(partitionAssignment.replicas(), counts.getOrDefault(partitionAssignment.replicas(), 0) + 1); } - assertEquals(14, counts.get(Arrays.asList(0, 1))); - assertEquals(22, counts.get(Arrays.asList(0, 2))); - assertEquals(14, counts.get(Arrays.asList(0, 3))); - assertEquals(17, counts.get(Arrays.asList(1, 0))); - assertEquals(17, counts.get(Arrays.asList(1, 2))); - assertEquals(16, counts.get(Arrays.asList(1, 3))); - assertEquals(13, counts.get(Arrays.asList(2, 0))); - assertEquals(17, counts.get(Arrays.asList(2, 1))); - assertEquals(20, counts.get(Arrays.asList(2, 3))); - assertEquals(20, counts.get(Arrays.asList(3, 0))); - assertEquals(19, counts.get(Arrays.asList(3, 1))); - assertEquals(11, counts.get(Arrays.asList(3, 2))); + assertEquals(14, counts.get(List.of(0, 1))); + assertEquals(22, counts.get(List.of(0, 2))); + assertEquals(14, counts.get(List.of(0, 3))); + assertEquals(17, counts.get(List.of(1, 0))); + assertEquals(17, counts.get(List.of(1, 2))); + assertEquals(16, counts.get(List.of(1, 3))); + assertEquals(13, counts.get(List.of(2, 0))); + assertEquals(17, counts.get(List.of(2, 1))); + assertEquals(20, counts.get(List.of(2, 3))); + assertEquals(20, counts.get(List.of(3, 0))); + assertEquals(19, counts.get(List.of(3, 1))); + assertEquals(11, counts.get(List.of(3, 2))); } @Test public void testRackListAllBrokersFenced() { // ensure we can place N replicas on a rack when the rack has less than N brokers MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(0, Optional.empty(), true), new UsableBroker(1, Optional.empty(), true), new UsableBroker(2, Optional.empty(), true)).iterator()); assertEquals(3, rackList.numTotalBrokers()); assertEquals(0, rackList.numUnfencedBrokers()); - assertEquals(Collections.singletonList(Optional.empty()), rackList.rackNames()); + assertEquals(List.of(Optional.empty()), rackList.rackNames()); assertEquals("All brokers are currently fenced.", assertThrows(InvalidReplicationFactorException.class, () -> rackList.place(3)).getMessage()); @@ -284,7 +282,7 @@ public class StripedReplicaPlacerTest { @Test public void testRackListNotEnoughBrokers() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false)).iterator()); assertEquals("The target replication factor of 3 cannot be reached because only " + @@ -296,7 +294,7 @@ public class StripedReplicaPlacerTest { @Test public void testRackListNonPositiveReplicationFactor() { MockRandom random = new MockRandom(); - RackList rackList = new RackList(random, Arrays.asList( + RackList rackList = new RackList(random, List.of( new UsableBroker(11, Optional.of("1"), false), new UsableBroker(10, Optional.of("1"), false)).iterator()); assertEquals("Invalid replication factor -1: the replication factor must be positive.", diff --git a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java index ce6fb6e77c9..3551e2ce7b0 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/placement/TopicAssignmentTest.java @@ -21,8 +21,6 @@ import org.apache.kafka.common.Uuid; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import static org.apache.kafka.metadata.placement.PartitionAssignmentTest.partitionAssignment; @@ -33,9 +31,9 @@ public class TopicAssignmentTest { @Test public void testTopicAssignmentReplicas() { - List replicasP0 = Arrays.asList(0, 1, 2); - List replicasP1 = Arrays.asList(1, 2, 0); - List partitionAssignments = Arrays.asList( + List replicasP0 = List.of(0, 1, 2); + List replicasP1 = List.of(1, 2, 0); + List partitionAssignments = List.of( partitionAssignment(replicasP0), partitionAssignment(replicasP1) ); @@ -44,18 +42,18 @@ public class TopicAssignmentTest { @Test public void testConsistentEqualsAndHashCode() { - List topicAssignments = Arrays.asList( + List topicAssignments = List.of( new TopicAssignment( - Collections.singletonList( + List.of( partitionAssignment( - Arrays.asList(0, 1, 2) + List.of(0, 1, 2) ) ) ), new TopicAssignment( - Collections.singletonList( + List.of( partitionAssignment( - Arrays.asList(1, 2, 0) + List.of(1, 2, 0) ) ) ) @@ -77,13 +75,13 @@ public class TopicAssignmentTest { @Test public void testToString() { - List replicas = Arrays.asList(0, 1, 2); - List directories = Arrays.asList( + List replicas = List.of(0, 1, 2); + List directories = List.of( Uuid.fromString("v56qeYzNRrqNtXsxzcReog"), Uuid.fromString("MvUIAsOiRlSePeiBHdZrSQ"), Uuid.fromString("jUqCchHtTHqMxeVv4dw1RA") ); - List partitionAssignments = Collections.singletonList( + List partitionAssignments = List.of( new PartitionAssignment(replicas, directories::get) ); TopicAssignment topicAssignment = new TopicAssignment(partitionAssignments); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java index 9af9f4685cc..51675528e50 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/properties/MetaPropertiesEnsembleTest.java @@ -26,8 +26,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Files; import java.util.AbstractMap.SimpleImmutableEntry; -import java.util.Arrays; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -37,6 +35,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalInt; import java.util.Random; +import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -54,8 +53,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows; public final class MetaPropertiesEnsembleTest { private static final MetaPropertiesEnsemble FOO = new MetaPropertiesEnsemble( - new HashSet<>(Arrays.asList("/tmp/empty1", "/tmp/empty2")), - new HashSet<>(Collections.singletonList("/tmp/error3")), + new HashSet<>(List.of("/tmp/empty1", "/tmp/empty2")), + new HashSet<>(List.of("/tmp/error3")), Stream.of( new SimpleImmutableEntry<>("/tmp/dir4", new MetaProperties.Builder(). @@ -94,7 +93,7 @@ public final class MetaPropertiesEnsembleTest { @Test public void testEmptyLogDirsForFoo() { - assertEquals(new HashSet<>(Arrays.asList("/tmp/empty1", "/tmp/empty2")), + assertEquals(new HashSet<>(List.of("/tmp/empty1", "/tmp/empty2")), FOO.emptyLogDirs()); } @@ -105,7 +104,7 @@ public final class MetaPropertiesEnsembleTest { @Test public void testErrorLogDirsForFoo() { - assertEquals(new HashSet<>(Collections.singletonList("/tmp/error3")), FOO.errorLogDirs()); + assertEquals(new HashSet<>(List.of("/tmp/error3")), FOO.errorLogDirs()); } @Test @@ -115,7 +114,7 @@ public final class MetaPropertiesEnsembleTest { @Test public void testLogDirPropsForFoo() { - assertEquals(new HashSet<>(Arrays.asList("/tmp/dir4", "/tmp/dir5")), + assertEquals(new HashSet<>(List.of("/tmp/dir4", "/tmp/dir5")), FOO.logDirProps().keySet()); } @@ -226,9 +225,9 @@ public final class MetaPropertiesEnsembleTest { @Test public void testVerificationFailureOnLackOfMetadataLogDir() { MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble( - Collections.singleton("/tmp/foo1"), - Collections.emptySet(), - Collections.emptyMap(), + Set.of("/tmp/foo1"), + Set.of(), + Map.of(), Optional.empty()); assertEquals("No metadata log directory was specified.", assertThrows(RuntimeException.class, @@ -241,9 +240,9 @@ public final class MetaPropertiesEnsembleTest { @Test public void testVerificationFailureOnMetadataLogDirWithError() { MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble( - Collections.emptySet(), - Collections.singleton("/tmp/foo1"), - Collections.emptyMap(), + Set.of(), + Set.of("/tmp/foo1"), + Map.of(), Optional.of("/tmp/foo1")); assertEquals("Encountered I/O error in metadata log directory /tmp/foo1. Cannot continue.", assertThrows(RuntimeException.class, @@ -285,7 +284,7 @@ public final class MetaPropertiesEnsembleTest { public void testMetaPropertiesEnsembleLoadError() throws IOException { MetaPropertiesEnsemble.Loader loader = new MetaPropertiesEnsemble.Loader(); loader.addMetadataLogDir(createErrorLogDir()); - loader.addLogDirs(Collections.singletonList(createLogDir(new MetaProperties.Builder(). + loader.addLogDirs(List.of(createLogDir(new MetaProperties.Builder(). setVersion(MetaPropertiesVersion.V1). setClusterId("AtgGav8yQjiaJ3rTXE7VCA"). setNodeId(1). @@ -323,7 +322,7 @@ public final class MetaPropertiesEnsembleTest { static class MetaPropertiesMockRandom extends Random { private final AtomicInteger index = new AtomicInteger(0); - private final List results = Arrays.asList( + private final List results = List.of( 0L, 0L, 2336837413447398698L, @@ -393,7 +392,7 @@ public final class MetaPropertiesEnsembleTest { assertThrows(RuntimeException.class, copier::verify).getMessage()); } - private static final List SAMPLE_META_PROPS_LIST = Arrays.asList( + private static final List SAMPLE_META_PROPS_LIST = List.of( new MetaProperties.Builder(). setVersion(MetaPropertiesVersion.V1). setClusterId("AtgGav8yQjiaJ3rTXE7VCA"). @@ -436,7 +435,7 @@ public final class MetaPropertiesEnsembleTest { String dir0 = createLogDir(SAMPLE_META_PROPS_LIST.get(0)); loader.addMetadataLogDir(dir0); String dir1 = createLogDir(SAMPLE_META_PROPS_LIST.get(1)); - loader.addLogDirs(Arrays.asList(dir0, dir1)); + loader.addLogDirs(List.of(dir0, dir1)); MetaPropertiesEnsemble ensemble = loader.load(); MetaPropertiesEnsemble.Copier copier = new MetaPropertiesEnsemble.Copier(ensemble); copier.setLogDirProps(dir0, SAMPLE_META_PROPS_LIST.get(2)); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index 0706e4e738f..12161968b1f 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -51,12 +51,10 @@ import java.io.File; import java.io.IOException; import java.io.PrintStream; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.OptionalInt; -import java.util.stream.Collectors; import java.util.stream.Stream; import static org.apache.kafka.metadata.storage.ScramParserTest.TEST_SALT; @@ -132,7 +130,7 @@ public class FormatterTest { } List outputLines() { - return Arrays.asList(stream.toString().trim().split("\\r*\\n")); + return List.of(stream.toString().trim().split("\\r*\\n")); } } @@ -197,7 +195,7 @@ public class FormatterTest { @Test public void testOneDirectoryFormattedAndOthersNotFormatted() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { - testEnv.newFormatter().formatter.setDirectories(Arrays.asList(testEnv.directory(0))).run(); + testEnv.newFormatter().formatter.setDirectories(List.of(testEnv.directory(0))).run(); assertEquals("Log directory " + testEnv.directory(0) + " is already formatted. " + "Use --ignore-formatted to ignore this directory and format the others.", assertThrows(FormatterException.class, @@ -208,7 +206,7 @@ public class FormatterTest { @Test public void testOneDirectoryFormattedAndOthersNotFormattedWithIgnoreFormatted() throws Exception { try (TestEnv testEnv = new TestEnv(2)) { - testEnv.newFormatter().formatter.setDirectories(Arrays.asList(testEnv.directory(0))).run(); + testEnv.newFormatter().formatter.setDirectories(List.of(testEnv.directory(0))).run(); FormatterContext formatter2 = testEnv.newFormatter(); formatter2.formatter.setIgnoreFormatted(true); @@ -280,7 +278,7 @@ public class FormatterTest { try (TestEnv testEnv = new TestEnv(1)) { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_4_IV0); - formatter1.formatter.setScramArguments(Arrays.asList( + formatter1.formatter.setScramArguments(List.of( "SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," + "saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]", "SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," + @@ -296,7 +294,7 @@ public class FormatterTest { try (TestEnv testEnv = new TestEnv(1)) { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_8_IV0); - formatter1.formatter.setScramArguments(Arrays.asList( + formatter1.formatter.setScramArguments(List.of( "SCRAM-SHA-256=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," + "saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]", "SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," + @@ -310,10 +308,10 @@ public class FormatterTest { assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion()); List scramRecords = bootstrapMetadata.records().stream(). filter(r -> r.message() instanceof UserScramCredentialRecord). - collect(Collectors.toList()); + toList(); ScramFormatter scram256 = new ScramFormatter(ScramMechanism.SCRAM_SHA_256); ScramFormatter scram512 = new ScramFormatter(ScramMechanism.SCRAM_SHA_512); - assertEquals(Arrays.asList( + assertEquals(List.of( new ApiMessageAndVersion(new UserScramCredentialRecord(). setName("alice"). setMechanism(ScramMechanism.SCRAM_SHA_256.type()). @@ -390,7 +388,7 @@ public class FormatterTest { parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); formatter1.formatter.run(); assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); - assertEquals(Arrays.asList( + assertEquals(List.of( String.format("Formatting data directory %s with %s %s.", testEnv.directory(1), MetadataVersion.FEATURE_NAME, @@ -399,7 +397,7 @@ public class FormatterTest { testEnv.directory(0), MetadataVersion.FEATURE_NAME, MetadataVersion.latestTesting())), - formatter1.outputLines().stream().sorted().collect(Collectors.toList())); + formatter1.outputLines().stream().sorted().toList()); MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader(). addLogDirs(testEnv.directories). load(); @@ -500,7 +498,7 @@ public class FormatterTest { formatter1.formatter.run(); assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); - assertEquals(Arrays.asList( + assertEquals(List.of( String.format("Formatting data directory %s with %s %s.", testEnv.directory(1), MetadataVersion.FEATURE_NAME, @@ -509,7 +507,7 @@ public class FormatterTest { testEnv.directory(0), MetadataVersion.FEATURE_NAME, MetadataVersion.latestTesting())), - formatter1.outputLines().stream().sorted().collect(Collectors.toList())); + formatter1.outputLines().stream().sorted().toList()); MetaPropertiesEnsemble ensemble = new MetaPropertiesEnsemble.Loader(). addLogDirs(testEnv.directories). load(); diff --git a/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java b/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java index 0fbd4ef00ae..2c3be023246 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/util/RecordRedactorTest.java @@ -27,7 +27,6 @@ import org.apache.kafka.metadata.KafkaConfigSchema; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -44,7 +43,7 @@ public final class RecordRedactorTest { define("quux", ConfigDef.Type.PASSWORD, ConfigDef.Importance.HIGH, "quuux2 doc")); } - private static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, Collections.emptyMap()); + private static final KafkaConfigSchema SCHEMA = new KafkaConfigSchema(CONFIGS, Map.of()); private static final RecordRedactor REDACTOR = new RecordRedactor(SCHEMA); diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java index db974195493..4ebec345b65 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java @@ -48,7 +48,6 @@ import org.slf4j.LoggerFactory; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.IdentityHashMap; import java.util.Iterator; @@ -65,7 +64,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import java.util.stream.IntStream; /** @@ -573,7 +571,7 @@ public final class LocalLogManager implements RaftClient, listenerData.handleCommit( MemoryBatchReader.of( - Collections.singletonList( + List.of( Batch.data( entryOffset - batch.records.size() + 1, batch.leaderEpoch, @@ -810,7 +808,7 @@ public final class LocalLogManager implements RaftClient, public List> listeners() { final CompletableFuture>> future = new CompletableFuture<>(); eventQueue.append(() -> - future.complete(listeners.values().stream().map(l -> l.listener).collect(Collectors.toList())) + future.complete(listeners.values().stream().map(l -> l.listener).toList()) ); try { return future.get(); diff --git a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java index 3e1c77181da..b42f781e985 100644 --- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java +++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManagerTest.java @@ -25,10 +25,8 @@ import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.List; import java.util.OptionalInt; -import java.util.stream.Collectors; import static org.apache.kafka.metalog.MockMetaLogManagerListener.COMMIT; import static org.apache.kafka.metalog.MockMetaLogManagerListener.LAST_COMMITTED_OFFSET; @@ -141,7 +139,7 @@ public class LocalLogManagerTest { LocalLogManager activeLogManager = env.logManagers().get(leaderId); int epoch = activeLogManager.leaderAndEpoch().epoch(); - List messages = Arrays.asList( + List messages = List.of( new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(0), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(1), (short) 0), new ApiMessageAndVersion(new RegisterBrokerRecord().setBrokerId(2), (short) 0)); @@ -152,7 +150,7 @@ public class LocalLogManagerTest { } List listeners = env.logManagers().stream(). map(m -> (MockMetaLogManagerListener) m.listeners().get(0)). - collect(Collectors.toList()); + toList(); env.close(); for (MockMetaLogManagerListener listener : listeners) { List events = listener.serializedEvents();