MINOR: Clean up metadata module (#19069)

Given that now we support Java 17 on our brokers, this PR replace the
use of the following in metadata module:

Collections.singletonList() and Collections.emptyList() with List.of()
Collections.singletonMap() and Collections.emptyMap() with Map.of()
Collections.singleton() and Collections.emptySet() with Set.of()

Reviewers: David Arthur <mumrah@gmail.com>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Sanskar Jhajharia 2025-03-16 00:39:40 +05:30 committed by GitHub
parent 7bff678699
commit 766caaa551
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
118 changed files with 1439 additions and 1555 deletions

View File

@ -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<Uuid, int[]> topicMap = elrMembers.get(brokerId);
if (topicMap == null) {
topicMap = Collections.emptyMap();
topicMap = Map.of();
}
return new BrokersToIsrs.PartitionsOnReplicaIterator(topicMap, false);
}

View File

@ -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<Uuid, int[]> topicMap = isrMembers.get(brokerId);
if (topicMap == null) {
topicMap = Collections.emptyMap();
topicMap = Map.of();
}
return new PartitionsOnReplicaIterator(topicMap, leadersOnly);
}

View File

@ -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<ApiMessageAndVersion> newRecords = new ArrayList<>(newQuotaConfigs.size());
Map<String, Double> currentQuotas = clientQuotaData.containsKey(entity) ?
clientQuotaData.get(entity) : Collections.emptyMap();
clientQuotaData.get(entity) : Map.of();
for (Map.Entry<String, Double> entry : newQuotaConfigs.entrySet()) {
String key = entry.getKey();
Double newValue = entry.getValue();

View File

@ -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<ConfigResource> existenceChecker = __ -> { };
private Optional<AlterConfigPolicy> alterConfigPolicy = Optional.empty();
private ConfigurationValidator validator = ConfigurationValidator.NO_OP;
private Map<String, Object> staticConfig = Collections.emptyMap();
private Map<String, Object> 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<ApiMessageAndVersion> createClearElrRecordsAsNeeded(List<ApiMessageAndVersion> input) {
if (!featureControl.isElrFeatureEnabled()) {
return Collections.emptyList();
return List.of();
}
List<ApiMessageAndVersion> 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<ApiMessageAndVersion> recordsExplicitlyAltered = new ArrayList<>();
Map<String, String> currentConfigs = configData.get(configResource);
if (currentConfigs == null) {
currentConfigs = Collections.emptyMap();
currentConfigs = Map.of();
}
for (Entry<String, String> entry : newConfigs.entrySet()) {
String key = entry.getKey();
@ -544,7 +543,7 @@ public class ConfigurationControlManager {
Map<String, String> getConfigs(ConfigResource configResource) {
Map<String, String> 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<String, String> clusterConfig() {
Map<String, String> result = configData.get(DEFAULT_NODE);
return (result == null) ? Collections.emptyMap() : result;
return (result == null) ? Map.of() : result;
}
Map<String, String> currentControllerConfig() {
Map<String, String> result = configData.get(currentController);
return (result == null) ? Collections.emptyMap() : result;
return (result == null) ? Map.of() : result;
}
Map<String, String> currentTopicConfig(String topicName) {
Map<String, String> result = configData.get(new ConfigResource(Type.TOPIC, topicName));
return (result == null) ? Collections.emptyMap() : result;
return (result == null) ? Map.of() : result;
}
// Visible to test

View File

@ -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<T> {
}
public ControllerResult<T> withoutRecords() {
return new ControllerResult<>(Collections.emptyList(), response, false);
return new ControllerResult<>(List.of(), response, false);
}
public static <T> ControllerResult<T> atomicOf(List<ApiMessageAndVersion> records, T response) {

View File

@ -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,

View File

@ -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);
}

View File

@ -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<ApiMessageAndVersion> 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;
}

View File

@ -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

View File

@ -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> createTopicPolicy = Optional.empty();
private Optional<AlterConfigPolicy> alterConfigPolicy = Optional.empty();
private ConfigurationValidator configurationValidator = ConfigurationValidator.NO_OP;
private Map<String, Object> staticConfig = Collections.emptyMap();
private Map<String, Object> 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<String> 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<Uuid> 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<Uuid> 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<Map<ConfigResource, ApiError>> result =
@ -1920,7 +1919,7 @@ public final class QuorumController implements Controller {
Map<ConfigResource, Map<String, String>> newConfigs, boolean validateOnly
) {
if (newConfigs.isEmpty()) {
return CompletableFuture.completedFuture(Collections.emptyMap());
return CompletableFuture.completedFuture(Map.of());
}
return appendWriteEvent("legacyAlterConfigs", context.deadlineNs(), () -> {
ControllerResult<Map<ConfigResource, ApiError>> 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<Map<ClientQuotaEntity, ApiError>> 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(), () -> {

View File

@ -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<Integer> 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<Integer> 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<BrokerIdAndEpoch> 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<Integer> 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<Integer> 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();
}

View File

@ -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,

View File

@ -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,

View File

@ -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<Uuid, StandardAcl> acls;

View File

@ -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<String, Double> quotas;

View File

@ -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<ClientQuotaEntity, ClientQuotaImage> entities;

View File

@ -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<Integer, BrokerRegistration> brokers;

View File

@ -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);

View File

@ -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<ConfigResource, ConfigurationImage> data;
@ -70,7 +70,7 @@ public final class ConfigurationsImage {
if (configurationImage != null) {
return configurationImage.toMap();
} else {
return Collections.emptyMap();
return Map.of();
}
}

View File

@ -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

View File

@ -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()
);

View File

@ -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<ScramMechanism, Map<String, ScramCredentialData>> mechanisms;

View File

@ -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())
));
}
}

View File

@ -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());
}

View File

@ -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

View File

@ -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<String> childNames() {
return Collections.singletonList(AclsImageByIdNode.NAME);
return List.of(AclsImageByIdNode.NAME);
}
@Override

View File

@ -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<String> childNames() {
return Arrays.asList(ClusterImageBrokersNode.NAME, ClusterImageControllersNode.NAME);
return List.of(ClusterImageBrokersNode.NAME, ClusterImageControllersNode.NAME);
}
@Override

View File

@ -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<String> childNames() {
return Collections.emptyList();
return List.of();
}
/**

View File

@ -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<String> childNames() {
return Collections.singletonList("nextProducerId");
return List.of("nextProducerId");
}
@Override

View File

@ -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<String, ScramCredentialData> userData = image.mechanisms().get(mechanism);
return new ScramMechanismNode(userData == null ? Collections.emptyMap() : userData);
return new ScramMechanismNode(userData == null ? Map.of() : userData);
}
}

View File

@ -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<String> childNames() {
return Arrays.asList(TopicsImageByNameNode.NAME, TopicsImageByIdNode.NAME);
return List.of(TopicsImageByNameNode.NAME, TopicsImageByIdNode.NAME);
}
@Override

View File

@ -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<Integer, ControllerRegistration> controllers;
public ControllerRegistrationsPublisher() {
this.controllers = Collections.emptyMap();
this.controllers = Map.of();
}
@Override

View File

@ -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));

View File

@ -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) {

View File

@ -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<ConfigSynonym> synonyms = logConfigSynonyms.getOrDefault(configKey.name, emptyList());
List<ConfigSynonym> 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<ConfigSynonym> synonyms = logConfigSynonyms.getOrDefault(configKey.name, emptyList());
List<ConfigSynonym> 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);
}

View File

@ -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<Integer> isr() {
return isrWithBrokerEpoch.stream()
.map(BrokerState::brokerId)
.collect(Collectors.toList());
.toList();
}
public boolean equalsAllowStalePartitionEpoch(LeaderAndIsr other) {

View File

@ -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<String> getConfiguredSuperUsers(Map<String, ?> 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<String> result = new HashSet<>();
for (String value : values) {

View File

@ -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<AclOperation> IMPLIES_DESCRIBE = Collections.unmodifiableSet(
EnumSet.of(DESCRIBE, READ, WRITE, DELETE, ALTER));
private static final Set<AclOperation> 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<AclOperation> IMPLIES_DESCRIBE_CONFIGS = Collections.unmodifiableSet(
EnumSet.of(DESCRIBE_CONFIGS, ALTER_CONFIGS));
private static final Set<AclOperation> IMPLIES_DESCRIBE_CONFIGS =
Set.of(DESCRIBE_CONFIGS, ALTER_CONFIGS);
static AuthorizationResult findResult(Action action,
AuthorizableRequestContext requestContext,

View File

@ -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<ApiMessageAndVersion> records = Collections.singletonList(
List<ApiMessageAndVersion> records = List.of(
new ApiMessageAndVersion(new FeatureLevelRecord().
setName(MetadataVersion.FEATURE_NAME).
setFeatureLevel(metadataVersion.featureLevel()), (short) 0));

View File

@ -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()
);
}
}

View File

@ -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());
/**

View File

@ -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<String> scramArguments = Collections.emptyList();
private List<String> scramArguments = List.of();
/**
* The name of the initial controller listener.

View File

@ -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<ApiMessageAndVersion> messageBatch) {

View File

@ -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(),

View File

@ -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<Uuid, StandardAcl> acls = Collections.emptyMap();
Map<Uuid, StandardAcl> acls = Map.of();
@Override
public void setAclMutator(AclMutator aclMutator) {
@ -312,7 +310,7 @@ public class AclControlManagerTest {
assertFalse(manager.idToAcl().isEmpty());
ControllerResult<List<AclDeleteResult>> 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<List<AclCreateResult>> createResult = manager.createAcls(Collections.singletonList(aclBinding));
ControllerResult<List<AclCreateResult>> createResult = manager.createAcls(List.of(aclBinding));
Uuid id = ((AccessControlEntryRecord) createResult.records().get(0).message()).id();
assertEquals(1, createResult.records().size());
ControllerResult<List<AclDeleteResult>> deleteAclResultsAnyFilter = manager.deleteAcls(Collections.singletonList(AclBindingFilter.ANY));
ControllerResult<List<AclDeleteResult>> 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<List<AclDeleteResult>> deleteAclResultsSpecificFilter = manager.deleteAcls(Collections.singletonList(aclBinding.toFilter()));
ControllerResult<List<AclDeleteResult>> 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<List<AclDeleteResult>> deleteAclResultsBothFilters = manager.deleteAcls(Arrays.asList(AclBindingFilter.ANY, aclBinding.toFilter()));
ControllerResult<List<AclDeleteResult>> 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());

View File

@ -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);
}

View File

@ -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<TopicIdPartition> toSet(TopicIdPartition... partitions) {
return new HashSet<>(Arrays.asList(partitions));
return new HashSet<>(List.of(partitions));
}
private static Set<TopicIdPartition> toSet(BrokersToIsrs.PartitionsOnReplicaIterator iterator) {

View File

@ -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<TopicIdPartition> toSet(TopicIdPartition... partitions) {
return new HashSet<>(Arrays.asList(partitions));
return new HashSet<>(List.of(partitions));
}
private static Set<TopicIdPartition> toSet(PartitionsOnReplicaIterator iterator) {

View File

@ -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

View File

@ -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<Uuid> logDirs = metadataVersion.isDirectoryAssignmentSupported() ? asList(
List<Uuid> logDirs = metadataVersion.isDirectoryAssignmentSupported() ? List.of(
Uuid.fromString("63k9SN1nQOS0dFHSCIMA0A"),
Uuid.fromString("Vm1MjsOCR1OjDDydOsDbzg")
) : Collections.emptyList();
) : List.of();
ControllerResult<BrokerRegistrationReply> 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<Uuid> logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"));
List<Uuid> 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<Uuid> logDirs = asList(Uuid.fromString("yJGxmjfbQZSVFAlNM3uXZg"), Uuid.fromString("Mj3CW3OSRi29cFeNJlXuAQ"));
List<Uuid> 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)));

View File

@ -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<String, List<ConfigSynonym>> 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<ApiError> 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<Map<ConfigResource, ApiError>> 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<ApiMessageAndVersion> expectedRecords1 = asList(
List<ApiMessageAndVersion> 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<ApiError> 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<ApiError> 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<ApiError> 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());

View File

@ -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<String, VersionRange> features = QuorumFeatures.defaultSupportedFeatureMap(true);
features.putAll(rangeMap(args));
return new QuorumFeatures(0, features, emptyList());
return new QuorumFeatures(0, features, List.of());
}
private static Map<String, Short> 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<ApiError> 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<ApiError> 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<ApiError> 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<ApiError> 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<ApiError> 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<ApiError> 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)),

View File

@ -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());
}
}

View File

@ -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<Integer, Uuid> 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<Integer, Uuid> 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<Integer, Uuid> 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<Integer, Uuid> 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<Integer, Uuid> 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<ApiMessageAndVersion> 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());

View File

@ -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<PartitionReassignmentReplicas.CompletedReassignment> 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<PartitionReassignmentReplicas.CompletedReassignment> 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<PartitionReassignmentReplicas.CompletedReassignment> 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<PartitionReassignmentReplicas.CompletedReassignment> 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<PartitionReassignmentReplicas.CompletedReassignment> 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());
}
}

View File

@ -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());
}
}

View File

@ -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,

View File

@ -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).

View File

@ -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")

View File

@ -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<Map<ConfigResource, ApiError>> 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<Integer> allBrokers = Arrays.asList(1, 2, 3, 4, 5);
List<Integer> brokersToKeepUnfenced = Arrays.asList(1);
List<Integer> brokersToFence = Arrays.asList(2, 3, 4, 5);
List<Integer> allBrokers = List.of(1, 2, 3, 4, 5);
List<Integer> brokersToKeepUnfenced = List.of(1);
List<Integer> 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<Integer> allBrokers = Arrays.asList(1, 2, 3);
List<Integer> 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<Integer> allBrokers = Arrays.asList(1, 2, 3);
List<Integer> 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<Integer> allBrokers = Arrays.asList(1, 2, 3);
List<Integer> brokersToKeepUnfenced = Arrays.asList(1);
List<Integer> brokersToFence = Arrays.asList(2, 3);
List<Integer> allBrokers = List.of(1, 2, 3);
List<Integer> brokersToKeepUnfenced = List.of(1);
List<Integer> 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<Integer> allBrokers = Arrays.asList(1, 2, 3);
List<Integer> brokersToKeepUnfenced = Arrays.asList(1, 2);
List<Integer> brokersToFence = Collections.singletonList(3);
List<Integer> allBrokers = List.of(1, 2, 3);
List<Integer> brokersToKeepUnfenced = List.of(1, 2);
List<Integer> 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<TopicIdPartition> 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<ApiMessageAndVersion> generateTestRecords(Uuid fooId, Map<Integer, Long> 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<CreateTopicsResponseData> 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<Map<Uuid, ApiError>> deleteFuture =
controller.deleteTopics(context0, Collections.singletonList(Uuid.ZERO_UUID));
controller.deleteTopics(context0, List.of(Uuid.ZERO_UUID));
CompletableFuture<Map<String, ResultOrError<Uuid>>> findTopicIdsFuture =
controller.findTopicIds(context0, Collections.singletonList("foo"));
controller.findTopicIds(context0, List.of("foo"));
CompletableFuture<Map<Uuid, ResultOrError<String>>> findTopicNamesFuture =
controller.findTopicNames(context0, Collections.singletonList(Uuid.ZERO_UUID));
controller.findTopicNames(context0, List.of(Uuid.ZERO_UUID));
CompletableFuture<List<CreatePartitionsTopicResult>> createPartitionsFuture =
controller.createPartitions(context0, Collections.singletonList(
controller.createPartitions(context0, List.of(
new CreatePartitionsTopic()), false);
CompletableFuture<ElectLeadersResponseData> electLeadersFuture =
controller.electLeaders(context0, new ElectLeadersRequestData().setTimeoutMs(0).
@ -1293,7 +1290,7 @@ public class QuorumControllerTest {
CompletableFuture<AlterPartitionReassignmentsResponseData> alterReassignmentsFuture =
controller.alterPartitionReassignments(context0,
new AlterPartitionReassignmentsRequestData().setTimeoutMs(0).
setTopics(Collections.singletonList(new ReassignableTopic())));
setTopics(List.of(new ReassignableTopic())));
CompletableFuture<ListPartitionReassignmentsResponseData> listReassignmentsFuture =
controller.listPartitionReassignments(context0,
new ListPartitionReassignmentsRequestData().setTopics(null).setTimeoutMs(0));
@ -1334,15 +1331,15 @@ public class QuorumControllerTest {
CountDownLatch countDownLatch = pause(controller);
CompletableFuture<CreateTopicsResponseData> createFuture =
controller.createTopics(ANONYMOUS_CONTEXT, new CreateTopicsRequestData().
setTimeoutMs(120000), Collections.emptySet());
setTimeoutMs(120000), Set.of());
CompletableFuture<Map<Uuid, ApiError>> deleteFuture =
controller.deleteTopics(ANONYMOUS_CONTEXT, Collections.emptyList());
controller.deleteTopics(ANONYMOUS_CONTEXT, List.of());
CompletableFuture<Map<String, ResultOrError<Uuid>>> findTopicIdsFuture =
controller.findTopicIds(ANONYMOUS_CONTEXT, Collections.emptyList());
controller.findTopicIds(ANONYMOUS_CONTEXT, List.of());
CompletableFuture<Map<Uuid, ResultOrError<String>>> findTopicNamesFuture =
controller.findTopicNames(ANONYMOUS_CONTEXT, Collections.emptyList());
controller.findTopicNames(ANONYMOUS_CONTEXT, List.of());
CompletableFuture<List<CreatePartitionsTopicResult>> createPartitionsFuture =
controller.createPartitions(ANONYMOUS_CONTEXT, Collections.emptyList(), false);
controller.createPartitions(ANONYMOUS_CONTEXT, List.of(), false);
CompletableFuture<ElectLeadersResponseData> electLeadersFuture =
controller.electLeaders(ANONYMOUS_CONTEXT, new ElectLeadersRequestData());
CompletableFuture<AlterPartitionReassignmentsResponseData> 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<Void> 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<ConfigResource, Collection<String>> configs = Collections.singletonMap(
Map<ConfigResource, Collection<String>> configs = Map.of(
defaultBrokerResource,
Collections.emptyList()
List.of()
);
Map<ConfigResource, ResultOrError<Map<String, String>>> results =
@ -1542,7 +1539,7 @@ public class QuorumControllerTest {
ResultOrError<Map<String, String>> 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<ApiMessageAndVersion> fromRecords) {
List<ImageDeltaPair<?, ?>> testMatrix = Arrays.asList(
List<ImageDeltaPair<?, ?>> 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<Void> 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(

View File

@ -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<Integer> nodeIds = IntStream.range(0, numControllers).boxed().collect(Collectors.toList());
List<Integer> 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));

View File

@ -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() {

View File

@ -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();
}

View File

@ -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).

View File

@ -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<String> expected = new HashSet<>(Arrays.asList(
HashSet<String> 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();
}

View File

@ -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();

View File

@ -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<ClientQuotaEntity, ClientQuotaImage> entities1 = new HashMap<>();
Map<String, String> fooUser = Collections.singletonMap(ClientQuotaEntity.USER, "foo");
Map<String, Double> fooUserQuotas = Collections.singletonMap(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 123.0);
Map<String, String> fooUser = Map.of(ClientQuotaEntity.USER, "foo");
Map<String, Double> fooUserQuotas = Map.of(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, 123.0);
entities1.put(new ClientQuotaEntity(fooUser), new ClientQuotaImage(fooUserQuotas));
Map<String, String> barUserAndIp = new HashMap<>();
barUserAndIp.put(ClientQuotaEntity.USER, "bar");
barUserAndIp.put(ClientQuotaEntity.IP, "127.0.0.1");
Map<String, Double> barUserAndIpQuotas = Collections.singletonMap(QuotaConfig.CONSUMER_BYTE_RATE_OVERRIDE_CONFIG, 456.0);
Map<String, Double> 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()));

View File

@ -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<String> lossString = new AtomicReference<>("");
testImage.write(writer, new ImageWriterOptions.Builder(MetadataVersion.IBP_3_6_IV2).

View File

@ -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<ConfigResource, ConfigurationImage> map2 = new HashMap<>();
Map<String, String> broker1Map2 = Collections.singletonMap("barfoo", "bazfoo");
Map<String, String> broker1Map2 = Map.of("barfoo", "bazfoo");
map2.put(new ConfigResource(BROKER, "1"),
new ConfigurationImage(new ConfigResource(BROKER, "1"), broker1Map2));
Map<String, String> broker2Map = Collections.singletonMap("foo", "bar");
Map<String, String> broker2Map = Map.of("foo", "bar");
map2.put(new ConfigResource(BROKER, "2"), new ConfigurationImage(new ConfigResource(BROKER, "2"), broker2Map));
IMAGE2 = new ConfigurationsImage(map2);
}

View File

@ -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<ApiMessageAndVersion>
private boolean closed = false;
public List<List<ApiMessageAndVersion>> batches() {
List<List<ApiMessageAndVersion>> result = new ArrayList<>();
for (List<ApiMessageAndVersion> batch : batches) {
result.add(Collections.unmodifiableList(batch));
}
return Collections.unmodifiableList(result);
return batches.stream().map(List::copyOf).toList();
}
public FakeSnapshotWriter() {

View File

@ -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());
}
}

View File

@ -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<String, Short> map3 = Collections.singletonMap("bar", (short) 1);
Map<String, Short> 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());

View File

@ -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<ApiMessageAndVersion> TEST_RECORDS = Arrays.asList(
static final List<ApiMessageAndVersion> 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))
);
}

View File

@ -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,

View File

@ -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<TopicImage> topics2 = Arrays.asList(
List<TopicImage> 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<ApiMessageAndVersion> imageRecords = getImageRecords(image);
@ -337,7 +336,7 @@ public class TopicsImageTest {
List<ApiMessageAndVersion> 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<ApiMessageAndVersion> 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<Uuid> uuids = new HashSet<>();
map.values().iterator().forEachRemaining(uuids::add);
HashSet<Uuid> expectedUuids = new HashSet<>(Arrays.asList(
HashSet<Uuid> 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<String> names = new HashSet<>();
map.values().iterator().forEachRemaining(names::add);
HashSet<String> expectedNames = new HashSet<>(Arrays.asList("foo", "bar"));
HashSet<String> 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"));

View File

@ -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<ApiMessageAndVersion> noOpRecords(int n) {
return IntStream.range(0, n)
.mapToObj(__ -> new ApiMessageAndVersion(new NoOpRecord(), (short) 0))
.collect(Collectors.toList());
.toList();
}

View File

@ -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<MockPublisher> publishers = asList(new MockPublisher("a"),
List<MockPublisher> 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<Batch<ApiMessageAndVersion>> 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<MockPublisher> publishers = asList(new MockPublisher("a"),
List<MockPublisher> 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<MockPublisher> publishers = singletonList(new MockPublisher());
List<MockPublisher> 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<ApiMessageAndVersion> records
) {
return new MockBatchReader(batchBaseOffset,
singletonList(newBatch(batchBaseOffset, epoch, records)));
List.of(newBatch(batchBaseOffset, epoch, records)));
}
static Batch<ApiMessageAndVersion> newBatch(
@ -472,7 +468,7 @@ public class MetadataLoaderTest {
public void testLoadEmptyBatch() throws Exception {
MockFaultHandler faultHandler = new MockFaultHandler("testLoadEmptyBatch");
MockTime time = new MockTime();
List<MockPublisher> publishers = singletonList(new MockPublisher());
List<MockPublisher> 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<MockPublisher> publishers = asList(new MockPublisher("a"),
List<MockPublisher> 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<MockPublisher> publishers = asList(new MockPublisher("a"),
List<MockPublisher> publishers = List.of(new MockPublisher("a"),
new MockPublisher("b"));
AtomicReference<OptionalLong> 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<MockPublisher> publishers = singletonList(new MockPublisher("a"));
List<MockPublisher> 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<MockPublisher> publishers = singletonList(publisher);
List<MockPublisher> 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<MockPublisher> publishers = singletonList(publisher);
List<MockPublisher> 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<MockPublisher> publishers = singletonList(publisher);
List<MockPublisher> 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)

View File

@ -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();
}

View File

@ -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());
}
}

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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<String, String> configs = new HashMap<>();

View File

@ -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<ConfigResource, ConfigurationImage> 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<String> 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<ConfigResource> 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, ""),

View File

@ -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

View File

@ -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;

View File

@ -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()));

View File

@ -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());

View File

@ -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",

View File

@ -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());
}
}

View File

@ -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, "")));
}
}

View File

@ -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());
}
}

View File

@ -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

View File

@ -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<BrokerRegistration> REGISTRATIONS = Arrays.asList(
private static final List<BrokerRegistration> 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")

View File

@ -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 <K, V> Map<K, V> doubleMap(K k1, V v1, K k2, V v2) {
HashMap<K, V> map = new HashMap<>();
LinkedHashMap<K, V> map = new LinkedHashMap<>();
map.put(k1, v1);
map.put(k2, v2);
return Collections.unmodifiableMap(map);
return map;
}
private static final List<ControllerRegistration> REGISTRATIONS = Arrays.asList(
private static final List<ControllerRegistration> 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()
);

View File

@ -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<String> UUID = Arrays.asList(
private static final List<String> UUID = List.of(
Uuid.randomUuid().toString(),
Uuid.randomUuid().toString(),
Uuid.randomUuid().toString());
private static final List<KafkaPrincipal> EMPTYRENEWERS = Collections.emptyList();
private static final List<KafkaPrincipal> EMPTYRENEWERS = List.of();
private static final List<TokenInformation> TOKENINFORMATION = Arrays.asList(
private static final List<TokenInformation> 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> DELEGATIONTOKENDATA = Arrays.asList(
private static final List<DelegationTokenData> DELEGATIONTOKENDATA = List.of(
new DelegationTokenData(TOKENINFORMATION.get(0)),
new DelegationTokenData(TOKENINFORMATION.get(1)),
new DelegationTokenData(TOKENINFORMATION.get(2)));

View File

@ -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);

View File

@ -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<String, List<ConfigSynonym>> 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<String, ConfigEntry> 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,

View File

@ -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));
}

Some files were not shown because too many files have changed in this diff Show More