mirror of https://github.com/apache/kafka.git
KAFKA-17759 Remove Utils.mkSet (#17460)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
b2380d7bf6
commit
b03fe66cfe
|
@ -18,7 +18,6 @@ package org.apache.kafka.common.config;
|
|||
|
||||
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
|
||||
import org.apache.kafka.common.utils.Java;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -157,7 +156,7 @@ public class SslConfigs {
|
|||
.define(SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG, ConfigDef.Type.CLASS, null, ConfigDef.Importance.LOW, SslConfigs.SSL_ENGINE_FACTORY_CLASS_DOC);
|
||||
}
|
||||
|
||||
public static final Set<String> RECONFIGURABLE_CONFIGS = Utils.mkSet(
|
||||
public static final Set<String> RECONFIGURABLE_CONFIGS = Set.of(
|
||||
SslConfigs.SSL_KEYSTORE_TYPE_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG,
|
||||
|
@ -169,7 +168,7 @@ public class SslConfigs {
|
|||
SslConfigs.SSL_KEYSTORE_KEY_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG);
|
||||
|
||||
public static final Set<String> NON_RECONFIGURABLE_CONFIGS = Utils.mkSet(
|
||||
public static final Set<String> NON_RECONFIGURABLE_CONFIGS = Set.of(
|
||||
BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG,
|
||||
SslConfigs.SSL_PROTOCOL_CONFIG,
|
||||
SslConfigs.SSL_PROVIDER_CONFIG,
|
||||
|
|
|
@ -18,9 +18,7 @@ package org.apache.kafka.common.internals;
|
|||
|
||||
import org.apache.kafka.common.TopicPartition;
|
||||
import org.apache.kafka.common.errors.InvalidTopicException;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
|
@ -36,8 +34,7 @@ public class Topic {
|
|||
);
|
||||
public static final String LEGAL_CHARS = "[a-zA-Z0-9._-]";
|
||||
|
||||
private static final Set<String> INTERNAL_TOPICS = Collections.unmodifiableSet(
|
||||
Utils.mkSet(GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME));
|
||||
private static final Set<String> INTERNAL_TOPICS = Set.of(GROUP_METADATA_TOPIC_NAME, TRANSACTION_STATE_TOPIC_NAME, SHARE_GROUP_STATE_TOPIC_NAME);
|
||||
|
||||
private static final int MAX_NAME_LENGTH = 249;
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.KafkaException;
|
|||
import org.apache.kafka.common.MetricName;
|
||||
import org.apache.kafka.common.config.ConfigException;
|
||||
import org.apache.kafka.common.utils.Sanitizer;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -58,7 +57,7 @@ public class JmxReporter implements MetricsReporter {
|
|||
public static final String INCLUDE_CONFIG = METRICS_CONFIG_PREFIX + "include";
|
||||
|
||||
|
||||
public static final Set<String> RECONFIGURABLE_CONFIGS = Utils.mkSet(INCLUDE_CONFIG,
|
||||
public static final Set<String> RECONFIGURABLE_CONFIGS = Set.of(INCLUDE_CONFIG,
|
||||
EXCLUDE_CONFIG);
|
||||
|
||||
public static final String DEFAULT_INCLUDE = ".*";
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.apache.kafka.common.security.scram.internals.ScramMessages.ClientFirs
|
|||
import org.apache.kafka.common.security.scram.internals.ScramMessages.ServerFinalMessage;
|
||||
import org.apache.kafka.common.security.scram.internals.ScramMessages.ServerFirstMessage;
|
||||
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCredentialCallback;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -58,7 +57,7 @@ import javax.security.sasl.SaslServerFactory;
|
|||
public class ScramSaslServer implements SaslServer {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(ScramSaslServer.class);
|
||||
private static final Set<String> SUPPORTED_EXTENSIONS = Utils.mkSet(ScramLoginModule.TOKEN_AUTH_CONFIG);
|
||||
private static final Set<String> SUPPORTED_EXTENSIONS = Set.of(ScramLoginModule.TOKEN_AUTH_CONFIG);
|
||||
|
||||
enum State {
|
||||
RECEIVE_CLIENT_FIRST_MESSAGE,
|
||||
|
|
|
@ -773,20 +773,6 @@ public final class Utils {
|
|||
return existingBuffer;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a set
|
||||
* @param elems the elements
|
||||
* @param <T> the type of element
|
||||
* @return Set
|
||||
*/
|
||||
@SafeVarargs
|
||||
public static <T> Set<T> mkSet(T... elems) {
|
||||
Set<T> result = new HashSet<>((int) (elems.length / 0.75) + 1);
|
||||
for (T elem : elems)
|
||||
result.add(elem);
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a sorted set
|
||||
* @param elems the elements
|
||||
|
|
|
@ -38,7 +38,6 @@ import org.apache.kafka.common.requests.RequestTestUtils;
|
|||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.MockClusterResourceListener;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -937,7 +936,7 @@ public class MetadataTest {
|
|||
oldTopicPartitionCounts.put("oldValidTopic", 2);
|
||||
oldTopicPartitionCounts.put("keepValidTopic", 3);
|
||||
|
||||
retainTopics.set(Utils.mkSet(
|
||||
retainTopics.set(Set.of(
|
||||
"oldInvalidTopic",
|
||||
"keepInvalidTopic",
|
||||
"oldUnauthorizedTopic",
|
||||
|
@ -974,7 +973,7 @@ public class MetadataTest {
|
|||
newTopicPartitionCounts.put("keepValidTopic", 2);
|
||||
newTopicPartitionCounts.put("newValidTopic", 4);
|
||||
|
||||
retainTopics.set(Utils.mkSet(
|
||||
retainTopics.set(Set.of(
|
||||
"keepInvalidTopic",
|
||||
"newInvalidTopic",
|
||||
"keepUnauthorizedTopic",
|
||||
|
@ -1037,7 +1036,7 @@ public class MetadataTest {
|
|||
topicPartitionCounts.put("validTopic1", 2);
|
||||
topicPartitionCounts.put("validTopic2", 3);
|
||||
|
||||
retainTopics.set(Utils.mkSet(
|
||||
retainTopics.set(Set.of(
|
||||
"validTopic1",
|
||||
"validTopic2"));
|
||||
|
||||
|
@ -1058,7 +1057,7 @@ public class MetadataTest {
|
|||
|
||||
Cluster cluster = metadata.fetch();
|
||||
// We still have the topic, but it just doesn't have an ID.
|
||||
assertEquals(Utils.mkSet("validTopic1", "validTopic2"), cluster.topics());
|
||||
assertEquals(Set.of("validTopic1", "validTopic2"), cluster.topics());
|
||||
assertEquals(2, cluster.partitionsForTopic("validTopic1").size());
|
||||
assertEquals(new HashSet<>(topicIds.values()), new HashSet<>(cluster.topicIds()));
|
||||
assertEquals(Uuid.ZERO_UUID, cluster.topicId("validTopic1"));
|
||||
|
|
|
@ -1507,7 +1507,7 @@ public class KafkaAdminClientTest {
|
|||
String topicName0 = "test-0";
|
||||
Uuid topicId = Uuid.randomUuid();
|
||||
|
||||
int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
|
||||
int authorisedOperations = Utils.to32BitField(Set.of(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
|
||||
env.kafkaClient().prepareResponse(
|
||||
prepareDescribeClusterResponse(0,
|
||||
env.cluster().nodes(),
|
||||
|
@ -1543,7 +1543,7 @@ public class KafkaAdminClientTest {
|
|||
String topicName0 = "test-0";
|
||||
Uuid topicId = Uuid.randomUuid();
|
||||
|
||||
int authorisedOperations = Utils.to32BitField(Utils.mkSet(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
|
||||
int authorisedOperations = Utils.to32BitField(Set.of(AclOperation.DESCRIBE.code(), AclOperation.ALTER.code()));
|
||||
env.kafkaClient().prepareResponse(
|
||||
prepareDescribeClusterResponse(0,
|
||||
env.cluster().nodes(),
|
||||
|
@ -3038,7 +3038,7 @@ public class KafkaAdminClientTest {
|
|||
assertTrue(listing.state().isPresent());
|
||||
}
|
||||
|
||||
assertEquals(Utils.mkSet("group-1", "group-2", "group-3"), groupIds);
|
||||
assertEquals(Set.of("group-1", "group-2", "group-3"), groupIds);
|
||||
assertEquals(1, result.errors().get().size());
|
||||
}
|
||||
}
|
||||
|
@ -4939,7 +4939,7 @@ public class KafkaAdminClientTest {
|
|||
assertTrue(listing.state().isPresent());
|
||||
}
|
||||
|
||||
assertEquals(Utils.mkSet("share-group-1", "share-group-2", "share-group-3", "share-group-4"), groupIds);
|
||||
assertEquals(Set.of("share-group-1", "share-group-2", "share-group-3", "share-group-4"), groupIds);
|
||||
assertEquals(1, result.errors().get().size());
|
||||
}
|
||||
}
|
||||
|
@ -6351,14 +6351,14 @@ public class KafkaAdminClientTest {
|
|||
public void testUpdateFeaturesDuringSuccess(short version) throws Exception {
|
||||
final Map<String, FeatureUpdate> updates = makeTestFeatureUpdates();
|
||||
// Only v1 and below specifies error codes per feature for NONE error.
|
||||
Set<String> features = version <= 1 ? updates.keySet() : Utils.mkSet();
|
||||
Set<String> features = version <= 1 ? updates.keySet() : Set.of();
|
||||
testUpdateFeatures(updates, ApiError.NONE, features);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateFeaturesTopLevelError() throws Exception {
|
||||
final Map<String, FeatureUpdate> updates = makeTestFeatureUpdates();
|
||||
testUpdateFeatures(updates, new ApiError(Errors.INVALID_REQUEST), Utils.mkSet());
|
||||
testUpdateFeatures(updates, new ApiError(Errors.INVALID_REQUEST), Set.of());
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
@ -6369,7 +6369,7 @@ public class KafkaAdminClientTest {
|
|||
request -> request instanceof UpdateFeaturesRequest,
|
||||
UpdateFeaturesResponse.createWithErrors(
|
||||
new ApiError(Errors.NOT_CONTROLLER),
|
||||
Utils.mkSet(),
|
||||
Set.of(),
|
||||
0),
|
||||
env.cluster().nodeById(0));
|
||||
final int controllerId = 1;
|
||||
|
@ -6378,7 +6378,7 @@ public class KafkaAdminClientTest {
|
|||
controllerId,
|
||||
Collections.emptyList()));
|
||||
// Only v1 and below specifies error codes per feature for NONE error.
|
||||
Set<String> features = version <= 1 ? Utils.mkSet("test_feature_1", "test_feature_2") : Utils.mkSet();
|
||||
Set<String> features = version <= 1 ? Set.of("test_feature_1", "test_feature_2") : Set.of();
|
||||
env.kafkaClient().prepareResponseFrom(
|
||||
request -> request instanceof UpdateFeaturesRequest,
|
||||
UpdateFeaturesResponse.createWithErrors(
|
||||
|
@ -7842,7 +7842,7 @@ public class KafkaAdminClientTest {
|
|||
new TransactionListing("bar", 98765L, TransactionState.PREPARE_ABORT),
|
||||
new TransactionListing("baz", 13579L, TransactionState.COMPLETE_COMMIT)
|
||||
);
|
||||
assertEquals(Utils.mkSet(0, 1, 2), env.cluster().nodes().stream().map(Node::id)
|
||||
assertEquals(Set.of(0, 1, 2), env.cluster().nodes().stream().map(Node::id)
|
||||
.collect(Collectors.toSet()));
|
||||
|
||||
env.cluster().nodes().forEach(node -> {
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.clients.admin;
|
|||
import org.apache.kafka.common.KafkaException;
|
||||
import org.apache.kafka.common.KafkaFuture;
|
||||
import org.apache.kafka.common.internals.KafkaFutureImpl;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
|
@ -73,7 +72,7 @@ public class ListTransactionsResultTest {
|
|||
Map<Integer, KafkaFuture<Collection<TransactionListing>>> resultBrokerFutures =
|
||||
result.byBrokerId().get();
|
||||
|
||||
assertEquals(Utils.mkSet(1, 2), resultBrokerFutures.keySet());
|
||||
assertEquals(Set.of(1, 2), resultBrokerFutures.keySet());
|
||||
assertEquals(broker1Listings, resultBrokerFutures.get(1).get());
|
||||
assertEquals(broker2Listings, resultBrokerFutures.get(2).get());
|
||||
assertEquals(broker1Listings, result.allByBrokerId().get().get(1));
|
||||
|
@ -108,7 +107,7 @@ public class ListTransactionsResultTest {
|
|||
result.byBrokerId().get();
|
||||
|
||||
// Ensure that the future for broker 1 completes successfully
|
||||
assertEquals(Utils.mkSet(1, 2), resultBrokerFutures.keySet());
|
||||
assertEquals(Set.of(1, 2), resultBrokerFutures.keySet());
|
||||
assertEquals(broker1Listings, resultBrokerFutures.get(1).get());
|
||||
|
||||
// Everything else should fail
|
||||
|
|
|
@ -33,11 +33,12 @@ import org.apache.kafka.common.utils.LogContext;
|
|||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
@ -56,9 +57,9 @@ public class AbortTransactionHandlerTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1,
|
||||
emptySet()));
|
||||
assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1,
|
||||
mkSet(new TopicPartition("foo", 1))));
|
||||
Set.of(new TopicPartition("foo", 1))));
|
||||
assertThrows(IllegalArgumentException.class, () -> handler.buildRequest(1,
|
||||
mkSet(topicPartition, new TopicPartition("foo", 1))));
|
||||
Set.of(topicPartition, new TopicPartition("foo", 1))));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -85,9 +86,9 @@ public class AbortTransactionHandlerTest {
|
|||
assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node,
|
||||
emptySet(), new WriteTxnMarkersResponse(response)));
|
||||
assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node,
|
||||
mkSet(new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response)));
|
||||
Set.of(new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response)));
|
||||
assertThrows(IllegalArgumentException.class, () -> handler.handleResponse(node,
|
||||
mkSet(topicPartition, new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response)));
|
||||
Set.of(topicPartition, new TopicPartition("foo", 1)), new WriteTxnMarkersResponse(response)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -52,7 +52,6 @@ import java.util.concurrent.ExecutionException;
|
|||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -72,14 +71,14 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo", "bar"), mapped("foo", 1, "bar", 2)
|
||||
Set.of("foo", "bar"), mapped("foo", 1, "bar", 2)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), completed("foo", 15L),
|
||||
mkSet("bar"), completed("bar", 30L)
|
||||
Set.of("foo"), completed("foo", 15L),
|
||||
Set.of("bar"), completed("bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -95,14 +94,14 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo"), mapped("foo", 1),
|
||||
mkSet("bar"), mapped("bar", 1)
|
||||
Set.of("foo"), mapped("foo", 1),
|
||||
Set.of("bar"), mapped("bar", 1)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
Set.of("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -124,14 +123,14 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo"), failedLookup("foo", keyLookupException),
|
||||
mkSet("bar"), mapped("bar", 1)
|
||||
Set.of("foo"), failedLookup("foo", keyLookupException),
|
||||
Set.of("bar"), mapped("bar", 1)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("bar"), completed("bar", 30L)
|
||||
Set.of("bar"), completed("bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -148,24 +147,24 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo"), emptyLookup(),
|
||||
mkSet("bar"), mapped("bar", 1)
|
||||
Set.of("foo"), emptyLookup(),
|
||||
Set.of("bar"), mapped("bar", 1)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, LookupResult<String>> fooRetry = map(
|
||||
mkSet("foo"), mapped("foo", 1)
|
||||
Set.of("foo"), mapped("foo", 1)
|
||||
);
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> barFulfillment = map(
|
||||
mkSet("bar"), completed("bar", 30L)
|
||||
Set.of("bar"), completed("bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(fooRetry, barFulfillment);
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fooFulfillment = map(
|
||||
mkSet("foo"), completed("foo", 15L)
|
||||
Set.of("foo"), completed("foo", 15L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fooFulfillment);
|
||||
|
@ -182,8 +181,8 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), completed("foo", 15L),
|
||||
mkSet("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
Set.of("foo"), completed("foo", 15L),
|
||||
Set.of("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -200,8 +199,8 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), failed("foo", new UnknownServerException()),
|
||||
mkSet("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
Set.of("foo"), failed("foo", new UnknownServerException()),
|
||||
Set.of("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -218,14 +217,14 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), completed("foo", 15L),
|
||||
mkSet("bar", "baz"), completed("bar", 30L)
|
||||
Set.of("foo"), completed("foo", 15L),
|
||||
Set.of("bar", "baz"), completed("bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> bazRetry = map(
|
||||
mkSet("baz"), completed("baz", 45L)
|
||||
Set.of("baz"), completed("baz", 45L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), bazRetry);
|
||||
|
@ -241,27 +240,27 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo"), mapped("foo", 0),
|
||||
mkSet("bar"), mapped("bar", 1)
|
||||
Set.of("foo"), mapped("foo", 0),
|
||||
Set.of("bar"), mapped("bar", 1)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), completed("foo", 15L),
|
||||
mkSet("bar"), unmapped("bar")
|
||||
Set.of("foo"), completed("foo", 15L),
|
||||
Set.of("bar"), unmapped("bar")
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
||||
Map<Set<String>, LookupResult<String>> barLookupRetry = map(
|
||||
mkSet("bar"), mapped("bar", 1)
|
||||
Set.of("bar"), mapped("bar", 1)
|
||||
);
|
||||
|
||||
ctx.poll(barLookupRetry, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> barFulfillRetry = map(
|
||||
mkSet("bar"), completed("bar", 30L)
|
||||
Set.of("bar"), completed("bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), barFulfillRetry);
|
||||
|
@ -278,8 +277,8 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillmentResults = map(
|
||||
mkSet("foo"), failed("foo", new UnsupportedVersionException("")),
|
||||
mkSet("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
Set.of("foo"), failed("foo", new UnsupportedVersionException("")),
|
||||
Set.of("bar", "baz"), completed("bar", 30L, "baz", 45L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillmentResults);
|
||||
|
@ -297,9 +296,9 @@ class AdminApiDriverTest {
|
|||
ctx.handler.addRetriableUnsupportedVersionKey("foo");
|
||||
// The mapped ApiResults are only used in the onResponse/handleResponse path - anything that needs
|
||||
// to be handled in the onFailure path needs to be manually set up later.
|
||||
ctx.handler.expectRequest(mkSet("foo"), failed("foo", new UnsupportedVersionException("")));
|
||||
ctx.handler.expectRequest(mkSet("bar"), failed("bar", new UnsupportedVersionException("")));
|
||||
ctx.handler.expectRequest(mkSet("baz"), completed("baz", 45L));
|
||||
ctx.handler.expectRequest(Set.of("foo"), failed("foo", new UnsupportedVersionException("")));
|
||||
ctx.handler.expectRequest(Set.of("bar"), failed("bar", new UnsupportedVersionException("")));
|
||||
ctx.handler.expectRequest(Set.of("baz"), completed("baz", 45L));
|
||||
// Setting up specific fulfillment stage executions requires polling the driver in order to obtain
|
||||
// the request specs needed for the onResponse/onFailure callbacks.
|
||||
List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
|
||||
|
@ -317,7 +316,7 @@ class AdminApiDriverTest {
|
|||
});
|
||||
// Verify retry for "foo" but not for "bar" or "baz"
|
||||
ctx.poll(emptyMap(), map(
|
||||
mkSet("foo"), failed("foo", new UnsupportedVersionException(""))
|
||||
Set.of("foo"), failed("foo", new UnsupportedVersionException(""))
|
||||
));
|
||||
ctx.poll(emptyMap(), emptyMap());
|
||||
}
|
||||
|
@ -330,26 +329,26 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
Map<Set<String>, LookupResult<String>> lookupRequests = map(
|
||||
mkSet("foo", "bar"), mapped("foo", 1, "bar", 2)
|
||||
Set.of("foo", "bar"), mapped("foo", 1, "bar", 2)
|
||||
);
|
||||
|
||||
ctx.poll(lookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fulfillment = map(
|
||||
mkSet("foo"), unmapped("foo"),
|
||||
mkSet("bar"), unmapped("bar")
|
||||
Set.of("foo"), unmapped("foo"),
|
||||
Set.of("bar"), unmapped("bar")
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), fulfillment);
|
||||
|
||||
Map<Set<String>, LookupResult<String>> retryLookupRequests = map(
|
||||
mkSet("foo", "bar"), mapped("foo", 3, "bar", 3)
|
||||
Set.of("foo", "bar"), mapped("foo", 3, "bar", 3)
|
||||
);
|
||||
|
||||
ctx.poll(retryLookupRequests, emptyMap());
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> retryFulfillment = map(
|
||||
mkSet("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
Set.of("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
);
|
||||
|
||||
ctx.poll(emptyMap(), retryFulfillment);
|
||||
|
@ -366,13 +365,13 @@ class AdminApiDriverTest {
|
|||
int initialLeaderId = 1;
|
||||
|
||||
Map<Set<String>, LookupResult<String>> initialLookup = map(
|
||||
mkSet("foo"), mapped("foo", initialLeaderId)
|
||||
Set.of("foo"), mapped("foo", initialLeaderId)
|
||||
);
|
||||
|
||||
ctx.poll(initialLookup, emptyMap());
|
||||
assertMappedKey(ctx, "foo", initialLeaderId);
|
||||
|
||||
ctx.handler.expectRequest(mkSet("foo"), completed("foo", 15L));
|
||||
ctx.handler.expectRequest(Set.of("foo"), completed("foo", 15L));
|
||||
|
||||
List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
|
||||
assertEquals(1, requestSpecs.size());
|
||||
|
@ -385,7 +384,7 @@ class AdminApiDriverTest {
|
|||
|
||||
int retryLeaderId = 2;
|
||||
|
||||
ctx.lookupStrategy().expectLookup(mkSet("foo"), mapped("foo", retryLeaderId));
|
||||
ctx.lookupStrategy().expectLookup(Set.of("foo"), mapped("foo", retryLeaderId));
|
||||
List<RequestSpec<String>> retryLookupSpecs = ctx.driver.poll();
|
||||
assertEquals(1, retryLookupSpecs.size());
|
||||
|
||||
|
@ -450,23 +449,23 @@ class AdminApiDriverTest {
|
|||
// fulfillment request for the static key
|
||||
LookupResult<String> lookupResult = mapped("foo", 1);
|
||||
ctx.lookupStrategy().expectLookup(
|
||||
mkSet("foo"), lookupResult
|
||||
Set.of("foo"), lookupResult
|
||||
);
|
||||
ctx.handler.expectRequest(
|
||||
mkSet("bar"), completed("bar", 10L)
|
||||
Set.of("bar"), completed("bar", 10L)
|
||||
);
|
||||
|
||||
List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
|
||||
assertEquals(2, requestSpecs.size());
|
||||
|
||||
RequestSpec<String> lookupSpec = requestSpecs.get(0);
|
||||
assertEquals(mkSet("foo"), lookupSpec.keys);
|
||||
assertEquals(Set.of("foo"), lookupSpec.keys);
|
||||
ctx.assertLookupResponse(lookupSpec, lookupResult);
|
||||
|
||||
// Receive a disconnect from the fulfillment request so that
|
||||
// we have an opportunity to coalesce the keys.
|
||||
RequestSpec<String> fulfillmentSpec = requestSpecs.get(1);
|
||||
assertEquals(mkSet("bar"), fulfillmentSpec.keys);
|
||||
assertEquals(Set.of("bar"), fulfillmentSpec.keys);
|
||||
ctx.driver.onFailure(ctx.time.milliseconds(), fulfillmentSpec, new DisconnectException());
|
||||
|
||||
// Now we should get two fulfillment requests. One of them will
|
||||
|
@ -474,28 +473,28 @@ class AdminApiDriverTest {
|
|||
// should contain the single dynamic key for broker 0.
|
||||
ctx.handler.reset();
|
||||
ctx.handler.expectRequest(
|
||||
mkSet("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
Set.of("foo", "bar"), completed("foo", 15L, "bar", 30L)
|
||||
);
|
||||
|
||||
List<RequestSpec<String>> coalescedSpecs = ctx.driver.poll();
|
||||
assertEquals(1, coalescedSpecs.size());
|
||||
RequestSpec<String> coalescedSpec = coalescedSpecs.get(0);
|
||||
assertEquals(mkSet("foo", "bar"), coalescedSpec.keys);
|
||||
assertEquals(Set.of("foo", "bar"), coalescedSpec.keys);
|
||||
|
||||
// Disconnect in order to ensure that only the dynamic key is unmapped.
|
||||
// Then complete the remaining requests.
|
||||
ctx.driver.onFailure(ctx.time.milliseconds(), coalescedSpec, new DisconnectException());
|
||||
|
||||
Map<Set<String>, LookupResult<String>> fooLookupRetry = map(
|
||||
mkSet("foo"), mapped("foo", 3)
|
||||
Set.of("foo"), mapped("foo", 3)
|
||||
);
|
||||
Map<Set<String>, ApiResult<String, Long>> barFulfillmentRetry = map(
|
||||
mkSet("bar"), completed("bar", 30L)
|
||||
Set.of("bar"), completed("bar", 30L)
|
||||
);
|
||||
ctx.poll(fooLookupRetry, barFulfillmentRetry);
|
||||
|
||||
Map<Set<String>, ApiResult<String, Long>> fooFulfillmentRetry = map(
|
||||
mkSet("foo"), completed("foo", 15L)
|
||||
Set.of("foo"), completed("foo", 15L)
|
||||
);
|
||||
ctx.poll(emptyMap(), fooFulfillmentRetry);
|
||||
ctx.poll(emptyMap(), emptyMap());
|
||||
|
@ -508,7 +507,7 @@ class AdminApiDriverTest {
|
|||
));
|
||||
|
||||
LookupResult<String> emptyLookup = emptyLookup();
|
||||
ctx.lookupStrategy().expectLookup(mkSet("foo"), emptyLookup);
|
||||
ctx.lookupStrategy().expectLookup(Set.of("foo"), emptyLookup);
|
||||
|
||||
List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
|
||||
assertEquals(1, requestSpecs.size());
|
||||
|
@ -531,7 +530,7 @@ class AdminApiDriverTest {
|
|||
TestContext ctx = TestContext.staticMapped(map("foo", 0));
|
||||
|
||||
ApiResult<String, Long> emptyFulfillment = emptyFulfillment();
|
||||
ctx.handler.expectRequest(mkSet("foo"), emptyFulfillment);
|
||||
ctx.handler.expectRequest(Set.of("foo"), emptyFulfillment);
|
||||
|
||||
List<RequestSpec<String>> requestSpecs = ctx.driver.poll();
|
||||
assertEquals(1, requestSpecs.size());
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.kafka.common.requests.MetadataRequest;
|
|||
import org.apache.kafka.common.requests.MetadataResponse;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -112,7 +111,7 @@ public class AllBrokersStrategyIntegrationTest {
|
|||
assertEquals(1, lookupSpecs.size());
|
||||
AdminApiDriver.RequestSpec<AllBrokersStrategy.BrokerKey> lookupSpec = lookupSpecs.get(0);
|
||||
|
||||
Set<Integer> brokerIds = Utils.mkSet(1, 2);
|
||||
Set<Integer> brokerIds = Set.of(1, 2);
|
||||
driver.onResponse(time.milliseconds(), lookupSpec, responseWithBrokers(brokerIds), Node.noNode());
|
||||
assertTrue(result.all().isDone());
|
||||
|
||||
|
|
|
@ -29,7 +29,6 @@ import java.util.HashSet;
|
|||
import java.util.OptionalInt;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
|
@ -48,9 +47,9 @@ class AllBrokersStrategyTest {
|
|||
AllBrokersStrategy strategy = new AllBrokersStrategy(logContext);
|
||||
AllBrokersStrategy.BrokerKey key1 = new AllBrokersStrategy.BrokerKey(OptionalInt.empty());
|
||||
AllBrokersStrategy.BrokerKey key2 = new AllBrokersStrategy.BrokerKey(OptionalInt.of(1));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(mkSet(key1)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(mkSet(key2)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(mkSet(key1, key2)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Set.of(key1)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Set.of(key2)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Set.of(key1, key2)));
|
||||
|
||||
Set<AllBrokersStrategy.BrokerKey> keys = new HashSet<>(AllBrokersStrategy.LOOKUP_KEYS);
|
||||
keys.add(key2);
|
||||
|
@ -80,7 +79,7 @@ class AllBrokersStrategyTest {
|
|||
|
||||
assertEquals(Collections.emptyMap(), lookupResult.failedKeys);
|
||||
|
||||
Set<AllBrokersStrategy.BrokerKey> expectedMappedKeys = mkSet(
|
||||
Set<AllBrokersStrategy.BrokerKey> expectedMappedKeys = Set.of(
|
||||
new AllBrokersStrategy.BrokerKey(OptionalInt.of(1)),
|
||||
new AllBrokersStrategy.BrokerKey(OptionalInt.of(2))
|
||||
);
|
||||
|
@ -113,9 +112,9 @@ class AllBrokersStrategyTest {
|
|||
AllBrokersStrategy.BrokerKey key2 = new AllBrokersStrategy.BrokerKey(OptionalInt.of(1));
|
||||
MetadataResponse response = new MetadataResponse(new MetadataResponseData(), ApiKeys.METADATA.latestVersion());
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(mkSet(key1), response));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(mkSet(key2), response));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(mkSet(key1, key2), response));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(Set.of(key1), response));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(Set.of(key2), response));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.handleResponse(Set.of(key1, key2), response));
|
||||
|
||||
Set<AllBrokersStrategy.BrokerKey> keys = new HashSet<>(AllBrokersStrategy.LOOKUP_KEYS);
|
||||
keys.add(key2);
|
||||
|
|
|
@ -36,7 +36,6 @@ import java.util.Set;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -82,7 +81,7 @@ public class CoordinatorStrategyTest {
|
|||
|
||||
CoordinatorKey group1 = CoordinatorKey.byGroupId("foo");
|
||||
CoordinatorKey group2 = CoordinatorKey.byGroupId("bar");
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(mkSet(group1, group2)));
|
||||
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(Set.of(group1, group2)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -124,7 +123,7 @@ public class CoordinatorStrategyTest {
|
|||
CoordinatorKey group1 = CoordinatorKey.byGroupId("foo");
|
||||
CoordinatorKey group2 = CoordinatorKey.byGroupId("bar");
|
||||
assertThrows(IllegalArgumentException.class, () ->
|
||||
strategy.handleResponse(mkSet(group1, group2), response));
|
||||
strategy.handleResponse(Set.of(group1, group2), response));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -48,7 +48,6 @@ import static java.util.Collections.emptyList;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
@ -74,7 +73,7 @@ public class DeleteRecordsHandlerTest {
|
|||
@Test
|
||||
public void testBuildRequestSimple() {
|
||||
DeleteRecordsHandler handler = new DeleteRecordsHandler(recordsToDelete, logContext, timeout);
|
||||
DeleteRecordsRequest request = handler.buildBatchedRequest(node1.id(), mkSet(t0p0, t0p1)).build();
|
||||
DeleteRecordsRequest request = handler.buildBatchedRequest(node1.id(), Set.of(t0p0, t0p1)).build();
|
||||
List<DeleteRecordsRequestData.DeleteRecordsTopic> topics = request.data().topics();
|
||||
assertEquals(1, topics.size());
|
||||
DeleteRecordsRequestData.DeleteRecordsTopic topic = topics.get(0);
|
||||
|
@ -233,10 +232,10 @@ public class DeleteRecordsHandlerTest {
|
|||
AdminApiLookupStrategy<TopicPartition> strategy = handler.lookupStrategy();
|
||||
assertInstanceOf(PartitionLeaderStrategy.class, strategy);
|
||||
PartitionLeaderStrategy specificStrategy = (PartitionLeaderStrategy) strategy;
|
||||
MetadataRequest request = specificStrategy.buildRequest(mkSet(t0p0, t0p1, t0p2, t0p3)).build();
|
||||
assertEquals(mkSet("t0"), new HashSet<>(request.topics()));
|
||||
MetadataRequest request = specificStrategy.buildRequest(Set.of(t0p0, t0p1, t0p2, t0p3)).build();
|
||||
assertEquals(Set.of("t0"), new HashSet<>(request.topics()));
|
||||
|
||||
Set<TopicPartition> tpSet = mkSet(t0p0, t0p1, t0p2, t0p3);
|
||||
Set<TopicPartition> tpSet = Set.of(t0p0, t0p1, t0p2, t0p3);
|
||||
LookupResult<TopicPartition> lookupResult = strategy.handleResponse(tpSet, metadataResponse);
|
||||
assertEquals(emptyMap(), lookupResult.failedKeys);
|
||||
assertEquals(tpSet, lookupResult.mappedKeys.keySet());
|
||||
|
@ -246,13 +245,13 @@ public class DeleteRecordsHandlerTest {
|
|||
|
||||
DeleteRecordsRequest deleteRequest = handler.buildBatchedRequest(node1.id(), partitionsPerBroker.get(node1.id())).build();
|
||||
assertEquals(2, deleteRequest.data().topics().get(0).partitions().size());
|
||||
assertEquals(mkSet(t0p0, t0p2),
|
||||
assertEquals(Set.of(t0p0, t0p2),
|
||||
deleteRequest.data().topics().get(0).partitions().stream()
|
||||
.map(drp -> new TopicPartition("t0", drp.partitionIndex()))
|
||||
.collect(Collectors.toSet()));
|
||||
deleteRequest = handler.buildBatchedRequest(node2.id(), partitionsPerBroker.get(node2.id())).build();
|
||||
assertEquals(2, deleteRequest.data().topics().get(0).partitions().size());
|
||||
assertEquals(mkSet(t0p1, t0p3),
|
||||
assertEquals(Set.of(t0p1, t0p3),
|
||||
deleteRequest.data().topics().get(0).partitions().stream()
|
||||
.map(drp -> new TopicPartition("t0", drp.partitionIndex()))
|
||||
.collect(Collectors.toSet()));
|
||||
|
|
|
@ -61,7 +61,6 @@ import static java.util.Collections.emptyList;
|
|||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -158,11 +157,11 @@ public class DescribeConsumerGroupsHandlerTest {
|
|||
Optional.of("instanceId"),
|
||||
"clientId",
|
||||
"host",
|
||||
new MemberAssignment(mkSet(
|
||||
new MemberAssignment(Set.of(
|
||||
new TopicPartition("foo", 0),
|
||||
new TopicPartition("bar", 1))
|
||||
),
|
||||
Optional.of(new MemberAssignment(mkSet(
|
||||
Optional.of(new MemberAssignment(Set.of(
|
||||
new TopicPartition("foo", 1),
|
||||
new TopicPartition("bar", 2)
|
||||
)))
|
||||
|
|
|
@ -51,7 +51,6 @@ import static java.util.Collections.emptyList;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
|
@ -70,7 +69,7 @@ public class DescribeProducersHandlerTest {
|
|||
@Test
|
||||
public void testBrokerIdSetInOptions() {
|
||||
int brokerId = 3;
|
||||
Set<TopicPartition> topicPartitions = mkSet(
|
||||
Set<TopicPartition> topicPartitions = Set.of(
|
||||
new TopicPartition("foo", 5),
|
||||
new TopicPartition("bar", 3),
|
||||
new TopicPartition("foo", 4)
|
||||
|
@ -89,7 +88,7 @@ public class DescribeProducersHandlerTest {
|
|||
|
||||
@Test
|
||||
public void testBrokerIdNotSetInOptions() {
|
||||
Set<TopicPartition> topicPartitions = mkSet(
|
||||
Set<TopicPartition> topicPartitions = Set.of(
|
||||
new TopicPartition("foo", 5),
|
||||
new TopicPartition("bar", 3),
|
||||
new TopicPartition("foo", 4)
|
||||
|
@ -108,7 +107,7 @@ public class DescribeProducersHandlerTest {
|
|||
|
||||
@Test
|
||||
public void testBuildRequest() {
|
||||
Set<TopicPartition> topicPartitions = mkSet(
|
||||
Set<TopicPartition> topicPartitions = Set.of(
|
||||
new TopicPartition("foo", 5),
|
||||
new TopicPartition("bar", 3),
|
||||
new TopicPartition("foo", 4)
|
||||
|
@ -123,13 +122,13 @@ public class DescribeProducersHandlerTest {
|
|||
|
||||
List<DescribeProducersRequestData.TopicRequest> topics = request.data.topics();
|
||||
|
||||
assertEquals(mkSet("foo", "bar"), topics.stream()
|
||||
assertEquals(Set.of("foo", "bar"), topics.stream()
|
||||
.map(DescribeProducersRequestData.TopicRequest::name)
|
||||
.collect(Collectors.toSet()));
|
||||
|
||||
topics.forEach(topic -> {
|
||||
Set<Integer> expectedTopicPartitions = "foo".equals(topic.name()) ?
|
||||
mkSet(4, 5) : mkSet(3);
|
||||
Set.of(4, 5) : Set.of(3);
|
||||
assertEquals(expectedTopicPartitions, new HashSet<>(topic.partitionIndexes()));
|
||||
});
|
||||
}
|
||||
|
@ -140,7 +139,7 @@ public class DescribeProducersHandlerTest {
|
|||
Throwable exception = assertFatalError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED);
|
||||
assertInstanceOf(TopicAuthorizationException.class, exception);
|
||||
TopicAuthorizationException authException = (TopicAuthorizationException) exception;
|
||||
assertEquals(mkSet("foo"), authException.unauthorizedTopics());
|
||||
assertEquals(Set.of("foo"), authException.unauthorizedTopics());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -149,7 +148,7 @@ public class DescribeProducersHandlerTest {
|
|||
Throwable exception = assertFatalError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION);
|
||||
assertInstanceOf(InvalidTopicException.class, exception);
|
||||
InvalidTopicException invalidTopicException = (InvalidTopicException) exception;
|
||||
assertEquals(mkSet("foo"), invalidTopicException.invalidTopics());
|
||||
assertEquals(Set.of("foo"), invalidTopicException.invalidTopics());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -184,7 +183,7 @@ public class DescribeProducersHandlerTest {
|
|||
handleResponseWithError(options, topicPartition, Errors.NOT_LEADER_OR_FOLLOWER);
|
||||
assertEquals(emptyMap(), result.completedKeys);
|
||||
assertEquals(emptyList(), result.unmappedKeys);
|
||||
assertEquals(mkSet(topicPartition), result.failedKeys.keySet());
|
||||
assertEquals(Set.of(topicPartition), result.failedKeys.keySet());
|
||||
Throwable exception = result.failedKeys.get(topicPartition);
|
||||
assertInstanceOf(NotLeaderOrFollowerException.class, exception);
|
||||
}
|
||||
|
@ -202,9 +201,9 @@ public class DescribeProducersHandlerTest {
|
|||
Node node = new Node(3, "host", 1);
|
||||
|
||||
ApiResult<TopicPartition, PartitionProducerState> result =
|
||||
handler.handleResponse(node, mkSet(topicPartition), response);
|
||||
handler.handleResponse(node, Set.of(topicPartition), response);
|
||||
|
||||
assertEquals(mkSet(topicPartition), result.completedKeys.keySet());
|
||||
assertEquals(Set.of(topicPartition), result.completedKeys.keySet());
|
||||
assertEquals(emptyMap(), result.failedKeys);
|
||||
assertEquals(emptyList(), result.unmappedKeys);
|
||||
|
||||
|
@ -231,7 +230,7 @@ public class DescribeProducersHandlerTest {
|
|||
new DescribeProducersOptions(), topicPartition, error);
|
||||
assertEquals(emptyMap(), result.completedKeys);
|
||||
assertEquals(emptyList(), result.unmappedKeys);
|
||||
assertEquals(mkSet(topicPartition), result.failedKeys.keySet());
|
||||
assertEquals(Set.of(topicPartition), result.failedKeys.keySet());
|
||||
return result.failedKeys.get(topicPartition);
|
||||
}
|
||||
|
||||
|
@ -243,7 +242,7 @@ public class DescribeProducersHandlerTest {
|
|||
DescribeProducersHandler handler = newHandler(options);
|
||||
DescribeProducersResponse response = buildResponseWithError(topicPartition, error);
|
||||
Node node = new Node(options.brokerId().orElse(3), "host", 1);
|
||||
return handler.handleResponse(node, mkSet(topicPartition), response);
|
||||
return handler.handleResponse(node, Set.of(topicPartition), response);
|
||||
}
|
||||
|
||||
private DescribeProducersResponse buildResponseWithError(
|
||||
|
|
|
@ -36,7 +36,6 @@ import static java.util.Arrays.asList;
|
|||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
|
||||
|
@ -50,12 +49,12 @@ public class DescribeTransactionsHandlerTest {
|
|||
String transactionalId2 = "bar";
|
||||
String transactionalId3 = "baz";
|
||||
|
||||
Set<String> transactionalIds = mkSet(transactionalId1, transactionalId2, transactionalId3);
|
||||
Set<String> transactionalIds = Set.of(transactionalId1, transactionalId2, transactionalId3);
|
||||
DescribeTransactionsHandler handler = new DescribeTransactionsHandler(logContext);
|
||||
|
||||
assertLookup(handler, transactionalIds);
|
||||
assertLookup(handler, mkSet(transactionalId1));
|
||||
assertLookup(handler, mkSet(transactionalId2, transactionalId3));
|
||||
assertLookup(handler, Set.of(transactionalId1));
|
||||
assertLookup(handler, Set.of(transactionalId2, transactionalId3));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -63,7 +62,7 @@ public class DescribeTransactionsHandlerTest {
|
|||
String transactionalId1 = "foo";
|
||||
String transactionalId2 = "bar";
|
||||
|
||||
Set<String> transactionalIds = mkSet(transactionalId1, transactionalId2);
|
||||
Set<String> transactionalIds = Set.of(transactionalId1, transactionalId2);
|
||||
DescribeTransactionsHandler handler = new DescribeTransactionsHandler(logContext);
|
||||
|
||||
DescribeTransactionsResponseData.TransactionState transactionState1 =
|
||||
|
@ -105,7 +104,7 @@ public class DescribeTransactionsHandlerTest {
|
|||
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
|
||||
ApiResult<CoordinatorKey, TransactionDescription> result = handleResponseError(handler, transactionalId, error);
|
||||
assertEquals(emptyList(), result.unmappedKeys);
|
||||
assertEquals(mkSet(key), result.failedKeys.keySet());
|
||||
assertEquals(Set.of(key), result.failedKeys.keySet());
|
||||
|
||||
Throwable throwable = result.failedKeys.get(key);
|
||||
assertInstanceOf(error.exception().getClass(), throwable);
|
||||
|
@ -138,7 +137,7 @@ public class DescribeTransactionsHandlerTest {
|
|||
Errors error
|
||||
) {
|
||||
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
|
||||
Set<CoordinatorKey> keys = mkSet(key);
|
||||
Set<CoordinatorKey> keys = Set.of(key);
|
||||
|
||||
DescribeTransactionsResponseData.TransactionState transactionState = new DescribeTransactionsResponseData.TransactionState()
|
||||
.setErrorCode(error.code())
|
||||
|
|
|
@ -34,7 +34,6 @@ import static java.util.Collections.emptyList;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singleton;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
|
||||
|
@ -47,7 +46,7 @@ public class FenceProducersHandlerTest {
|
|||
@Test
|
||||
public void testBuildRequest() {
|
||||
FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
|
||||
mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, requestTimeoutMs));
|
||||
Set.of("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, requestTimeoutMs));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -55,7 +54,7 @@ public class FenceProducersHandlerTest {
|
|||
final int optionsTimeoutMs = 50000;
|
||||
options.timeoutMs(optionsTimeoutMs);
|
||||
FenceProducersHandler handler = new FenceProducersHandler(options, logContext, requestTimeoutMs);
|
||||
mkSet("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, optionsTimeoutMs));
|
||||
Set.of("foo", "bar", "baz").forEach(transactionalId -> assertLookup(handler, transactionalId, optionsTimeoutMs));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -106,7 +105,7 @@ public class FenceProducersHandlerTest {
|
|||
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
|
||||
ApiResult<CoordinatorKey, ProducerIdAndEpoch> result = handleResponseError(handler, transactionalId, error);
|
||||
assertEquals(emptyList(), result.unmappedKeys);
|
||||
assertEquals(mkSet(key), result.failedKeys.keySet());
|
||||
assertEquals(Set.of(key), result.failedKeys.keySet());
|
||||
|
||||
Throwable throwable = result.failedKeys.get(key);
|
||||
assertInstanceOf(error.exception().getClass(), throwable);
|
||||
|
@ -139,7 +138,7 @@ public class FenceProducersHandlerTest {
|
|||
Errors error
|
||||
) {
|
||||
CoordinatorKey key = CoordinatorKey.byTransactionalId(transactionalId);
|
||||
Set<CoordinatorKey> keys = mkSet(key);
|
||||
Set<CoordinatorKey> keys = Set.of(key);
|
||||
|
||||
InitProducerIdResponse response = new InitProducerIdResponse(new InitProducerIdResponseData()
|
||||
.setErrorCode(error.code()));
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.kafka.common.requests.OffsetFetchRequest;
|
|||
import org.apache.kafka.common.requests.OffsetFetchResponse;
|
||||
import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
|
@ -100,10 +99,10 @@ public class ListConsumerGroupOffsetsHandlerTest {
|
|||
|
||||
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(requestMap, false, logContext);
|
||||
OffsetFetchRequest request1 = handler.buildBatchedRequest(coordinatorKeys(groupZero, groupOne, groupTwo)).build();
|
||||
assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups(request1));
|
||||
assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups(request1));
|
||||
|
||||
OffsetFetchRequest request2 = handler.buildBatchedRequest(coordinatorKeys(groupThree)).build();
|
||||
assertEquals(Utils.mkSet(groupThree), requestGroups(request2));
|
||||
assertEquals(Set.of(groupThree), requestGroups(request2));
|
||||
|
||||
Map<String, ListConsumerGroupOffsetsSpec> builtRequests = new HashMap<>();
|
||||
request1.groupIdsToPartitions().forEach((group, partitions) ->
|
||||
|
@ -137,7 +136,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
|
|||
ListConsumerGroupOffsetsHandler handler = new ListConsumerGroupOffsetsHandler(batchedRequestMap, false, logContext);
|
||||
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo));
|
||||
assertEquals(1, requests.size());
|
||||
assertEquals(Utils.mkSet(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build()));
|
||||
assertEquals(Set.of(groupZero, groupOne, groupTwo), requestGroups((OffsetFetchRequest) requests.iterator().next().request.build()));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -148,7 +147,7 @@ public class ListConsumerGroupOffsetsHandlerTest {
|
|||
Collection<RequestAndKeys<CoordinatorKey>> requests = handler.buildRequest(1, coordinatorKeys(groupZero, groupOne, groupTwo));
|
||||
assertEquals(3, requests.size());
|
||||
assertEquals(
|
||||
Utils.mkSet(Utils.mkSet(groupZero), Utils.mkSet(groupOne), Utils.mkSet(groupTwo)),
|
||||
Set.of(Set.of(groupZero), Set.of(groupOne), Set.of(groupTwo)),
|
||||
requests.stream().map(requestAndKey -> requestGroups((OffsetFetchRequest) requestAndKey.request.build())).collect(Collectors.toSet())
|
||||
);
|
||||
}
|
||||
|
|
|
@ -48,7 +48,6 @@ import static java.util.Collections.emptyList;
|
|||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
|
@ -81,7 +80,7 @@ public final class ListOffsetsHandlerTest {
|
|||
public void testBuildRequestSimple() {
|
||||
ListOffsetsHandler handler =
|
||||
new ListOffsetsHandler(offsetTimestampsByPartition, new ListOffsetsOptions(), logContext, defaultApiTimeoutMs);
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1)).build();
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1)).build();
|
||||
List<ListOffsetsTopic> topics = request.topics();
|
||||
assertEquals(1, topics.size());
|
||||
ListOffsetsTopic topic = topics.get(0);
|
||||
|
@ -120,22 +119,22 @@ public final class ListOffsetsHandlerTest {
|
|||
ListOffsetsHandler defaultOptionsHandler =
|
||||
new ListOffsetsHandler(offsetTimestampsByPartition, new ListOffsetsOptions(), logContext, defaultApiTimeoutMs);
|
||||
ListOffsetsRequest.Builder builder =
|
||||
defaultOptionsHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0));
|
||||
defaultOptionsHandler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1, t1p0));
|
||||
assertEquals(1, builder.oldestAllowedVersion());
|
||||
|
||||
ListOffsetsHandler readCommittedHandler =
|
||||
new ListOffsetsHandler(
|
||||
offsetTimestampsByPartition, new ListOffsetsOptions(IsolationLevel.READ_COMMITTED), logContext, defaultApiTimeoutMs);
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0));
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1, t1p0));
|
||||
assertEquals(2, builder.oldestAllowedVersion());
|
||||
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1));
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1, t1p0, t1p1));
|
||||
assertEquals(7, builder.oldestAllowedVersion());
|
||||
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1, t2p0));
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1, t1p0, t1p1, t2p0));
|
||||
assertEquals(8, builder.oldestAllowedVersion());
|
||||
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1, t1p0, t1p1, t2p0, t2p1));
|
||||
builder = readCommittedHandler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1, t1p0, t1p1, t2p0, t2p1));
|
||||
assertEquals(9, builder.oldestAllowedVersion());
|
||||
}
|
||||
|
||||
|
@ -258,7 +257,7 @@ public final class ListOffsetsHandlerTest {
|
|||
ListOffsetsOptions options = new ListOffsetsOptions();
|
||||
ListOffsetsHandler handler =
|
||||
new ListOffsetsHandler(offsetTimestampsByPartition, options, logContext, defaultApiTimeoutMs);
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1)).build();
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1)).build();
|
||||
assertEquals(defaultApiTimeoutMs, request.timeoutMs());
|
||||
}
|
||||
|
||||
|
@ -268,7 +267,7 @@ public final class ListOffsetsHandlerTest {
|
|||
ListOffsetsOptions options = new ListOffsetsOptions().timeoutMs(timeoutMs);
|
||||
ListOffsetsHandler handler =
|
||||
new ListOffsetsHandler(offsetTimestampsByPartition, options, logContext, defaultApiTimeoutMs);
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), mkSet(t0p0, t0p1)).build();
|
||||
ListOffsetsRequest request = handler.buildBatchedRequest(node.id(), Set.of(t0p0, t0p1)).build();
|
||||
assertEquals(timeoutMs, request.timeoutMs());
|
||||
}
|
||||
|
||||
|
|
|
@ -42,7 +42,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||
|
@ -55,7 +54,7 @@ public class PartitionLeaderStrategyTest {
|
|||
|
||||
@Test
|
||||
public void testBuildLookupRequest() {
|
||||
Set<TopicPartition> topicPartitions = mkSet(
|
||||
Set<TopicPartition> topicPartitions = Set.of(
|
||||
new TopicPartition("foo", 0),
|
||||
new TopicPartition("bar", 0),
|
||||
new TopicPartition("foo", 1),
|
||||
|
@ -65,13 +64,13 @@ public class PartitionLeaderStrategyTest {
|
|||
PartitionLeaderStrategy strategy = newStrategy();
|
||||
|
||||
MetadataRequest allRequest = strategy.buildRequest(topicPartitions).build();
|
||||
assertEquals(mkSet("foo", "bar", "baz"), new HashSet<>(allRequest.topics()));
|
||||
assertEquals(Set.of("foo", "bar", "baz"), new HashSet<>(allRequest.topics()));
|
||||
assertFalse(allRequest.allowAutoTopicCreation());
|
||||
|
||||
MetadataRequest partialRequest = strategy.buildRequest(
|
||||
topicPartitions.stream().filter(tp -> tp.topic().equals("foo")).collect(Collectors.toSet())
|
||||
).build();
|
||||
assertEquals(mkSet("foo"), new HashSet<>(partialRequest.topics()));
|
||||
assertEquals(Set.of("foo"), new HashSet<>(partialRequest.topics()));
|
||||
assertFalse(partialRequest.allowAutoTopicCreation());
|
||||
}
|
||||
|
||||
|
@ -81,7 +80,7 @@ public class PartitionLeaderStrategyTest {
|
|||
Throwable exception = assertFatalTopicError(topicPartition, Errors.TOPIC_AUTHORIZATION_FAILED);
|
||||
assertInstanceOf(TopicAuthorizationException.class, exception);
|
||||
TopicAuthorizationException authException = (TopicAuthorizationException) exception;
|
||||
assertEquals(mkSet("foo"), authException.unauthorizedTopics());
|
||||
assertEquals(Set.of("foo"), authException.unauthorizedTopics());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -90,7 +89,7 @@ public class PartitionLeaderStrategyTest {
|
|||
Throwable exception = assertFatalTopicError(topicPartition, Errors.INVALID_TOPIC_EXCEPTION);
|
||||
assertInstanceOf(InvalidTopicException.class, exception);
|
||||
InvalidTopicException invalidTopicException = (InvalidTopicException) exception;
|
||||
assertEquals(mkSet("foo"), invalidTopicException.invalidTopics());
|
||||
assertEquals(Set.of("foo"), invalidTopicException.invalidTopics());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -137,12 +136,12 @@ public class PartitionLeaderStrategyTest {
|
|||
topicPartition2, 1, Arrays.asList(2, 1, 3)));
|
||||
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(
|
||||
mkSet(topicPartition1, topicPartition2),
|
||||
Set.of(topicPartition1, topicPartition2),
|
||||
responseWithPartitionData(responsePartitions)
|
||||
);
|
||||
|
||||
assertEquals(emptyMap(), result.failedKeys);
|
||||
assertEquals(mkSet(topicPartition1, topicPartition2), result.mappedKeys.keySet());
|
||||
assertEquals(Set.of(topicPartition1, topicPartition2), result.mappedKeys.keySet());
|
||||
assertEquals(5, result.mappedKeys.get(topicPartition1));
|
||||
assertEquals(1, result.mappedKeys.get(topicPartition2));
|
||||
}
|
||||
|
@ -159,12 +158,12 @@ public class PartitionLeaderStrategyTest {
|
|||
unrequestedTopicPartition, Errors.UNKNOWN_SERVER_ERROR));
|
||||
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(
|
||||
mkSet(requestedTopicPartition),
|
||||
Set.of(requestedTopicPartition),
|
||||
responseWithPartitionData(responsePartitions)
|
||||
);
|
||||
|
||||
assertEquals(emptyMap(), result.failedKeys);
|
||||
assertEquals(mkSet(requestedTopicPartition), result.mappedKeys.keySet());
|
||||
assertEquals(Set.of(requestedTopicPartition), result.mappedKeys.keySet());
|
||||
assertEquals(5, result.mappedKeys.get(requestedTopicPartition));
|
||||
}
|
||||
|
||||
|
@ -178,7 +177,7 @@ public class PartitionLeaderStrategyTest {
|
|||
);
|
||||
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(
|
||||
mkSet(topicPartition),
|
||||
Set.of(topicPartition),
|
||||
responseWithPartitionData(responsePartitions)
|
||||
);
|
||||
|
||||
|
@ -232,7 +231,7 @@ public class PartitionLeaderStrategyTest {
|
|||
TopicPartition topicPartition,
|
||||
MetadataResponse response
|
||||
) {
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(mkSet(topicPartition), response);
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(Set.of(topicPartition), response);
|
||||
assertEquals(emptyMap(), result.failedKeys);
|
||||
assertEquals(emptyMap(), result.mappedKeys);
|
||||
}
|
||||
|
@ -241,8 +240,8 @@ public class PartitionLeaderStrategyTest {
|
|||
TopicPartition topicPartition,
|
||||
MetadataResponse response
|
||||
) {
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(mkSet(topicPartition), response);
|
||||
assertEquals(mkSet(topicPartition), result.failedKeys.keySet());
|
||||
LookupResult<TopicPartition> result = handleLookupResponse(Set.of(topicPartition), response);
|
||||
assertEquals(Set.of(topicPartition), result.failedKeys.keySet());
|
||||
return result.failedKeys.get(topicPartition);
|
||||
}
|
||||
|
||||
|
|
|
@ -1105,7 +1105,7 @@ public class KafkaConsumerTest {
|
|||
|
||||
// fetch offset for one topic
|
||||
client.prepareResponseFrom(offsetResponse(Utils.mkMap(Utils.mkEntry(tp0, offset1), Utils.mkEntry(tp1, -1L)), Errors.NONE), coordinator);
|
||||
final Map<TopicPartition, OffsetAndMetadata> committed = consumer.committed(Utils.mkSet(tp0, tp1));
|
||||
final Map<TopicPartition, OffsetAndMetadata> committed = consumer.committed(Set.of(tp0, tp1));
|
||||
assertEquals(2, committed.size());
|
||||
assertEquals(offset1, committed.get(tp0).offset());
|
||||
assertNull(committed.get(tp1));
|
||||
|
@ -1679,7 +1679,7 @@ public class KafkaConsumerTest {
|
|||
Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port());
|
||||
|
||||
// manual assignment
|
||||
Set<TopicPartition> partitions = Utils.mkSet(tp0, tp1);
|
||||
Set<TopicPartition> partitions = Set.of(tp0, tp1);
|
||||
consumer.assign(partitions);
|
||||
// verify consumer's assignment
|
||||
assertEquals(partitions, consumer.assignment());
|
||||
|
@ -2278,11 +2278,11 @@ public class KafkaConsumerTest {
|
|||
// a poll with non-zero milliseconds would complete three round-trips (discover, join, sync)
|
||||
TestUtils.waitForCondition(() -> {
|
||||
consumer.poll(Duration.ofMillis(100L));
|
||||
return consumer.assignment().equals(Utils.mkSet(tp0, t2p0));
|
||||
return consumer.assignment().equals(Set.of(tp0, t2p0));
|
||||
}, "Does not complete rebalance in time");
|
||||
|
||||
assertEquals(Utils.mkSet(topic, topic2), consumer.subscription());
|
||||
assertEquals(Utils.mkSet(tp0, t2p0), consumer.assignment());
|
||||
assertEquals(Set.of(topic, topic2), consumer.subscription());
|
||||
assertEquals(Set.of(tp0, t2p0), consumer.assignment());
|
||||
|
||||
// prepare a response of the outstanding fetch so that we have data available on the next poll
|
||||
Map<TopicPartition, FetchInfo> fetches1 = new HashMap<>();
|
||||
|
@ -2307,8 +2307,8 @@ public class KafkaConsumerTest {
|
|||
consumer.subscribe(Arrays.asList(topic, topic3), getConsumerRebalanceListener(consumer));
|
||||
|
||||
// verify that subscription has changed but assignment is still unchanged
|
||||
assertEquals(Utils.mkSet(topic, topic3), consumer.subscription());
|
||||
assertEquals(Utils.mkSet(tp0, t2p0), consumer.assignment());
|
||||
assertEquals(Set.of(topic, topic3), consumer.subscription());
|
||||
assertEquals(Set.of(tp0, t2p0), consumer.assignment());
|
||||
|
||||
// mock the offset commit response for to be revoked partitions
|
||||
Map<TopicPartition, Long> partitionOffsets1 = new HashMap<>();
|
||||
|
@ -2324,7 +2324,7 @@ public class KafkaConsumerTest {
|
|||
client.respondFrom(fetchResponse(fetches1), node);
|
||||
|
||||
// verify that the fetch still occurred as expected
|
||||
assertEquals(Utils.mkSet(topic, topic3), consumer.subscription());
|
||||
assertEquals(Set.of(topic, topic3), consumer.subscription());
|
||||
assertEquals(Collections.singleton(tp0), consumer.assignment());
|
||||
assertEquals(1, records.count());
|
||||
assertEquals(2L, consumer.position(tp0));
|
||||
|
@ -2340,7 +2340,7 @@ public class KafkaConsumerTest {
|
|||
records = consumer.poll(Duration.ZERO);
|
||||
|
||||
// should not finish the response yet
|
||||
assertEquals(Utils.mkSet(topic, topic3), consumer.subscription());
|
||||
assertEquals(Set.of(topic, topic3), consumer.subscription());
|
||||
assertEquals(Collections.singleton(tp0), consumer.assignment());
|
||||
assertEquals(1, records.count());
|
||||
assertEquals(3L, consumer.position(tp0));
|
||||
|
@ -2355,13 +2355,13 @@ public class KafkaConsumerTest {
|
|||
AtomicInteger count = new AtomicInteger(0);
|
||||
TestUtils.waitForCondition(() -> {
|
||||
ConsumerRecords<String, String> recs = consumer.poll(Duration.ofMillis(100L));
|
||||
return consumer.assignment().equals(Utils.mkSet(tp0, t3p0)) && count.addAndGet(recs.count()) == 1;
|
||||
return consumer.assignment().equals(Set.of(tp0, t3p0)) && count.addAndGet(recs.count()) == 1;
|
||||
|
||||
}, "Does not complete rebalance in time");
|
||||
|
||||
// should have t3 but not sent yet the t3 records
|
||||
assertEquals(Utils.mkSet(topic, topic3), consumer.subscription());
|
||||
assertEquals(Utils.mkSet(tp0, t3p0), consumer.assignment());
|
||||
assertEquals(Set.of(topic, topic3), consumer.subscription());
|
||||
assertEquals(Set.of(tp0, t3p0), consumer.assignment());
|
||||
assertEquals(4L, consumer.position(tp0));
|
||||
assertEquals(0L, consumer.position(t3p0));
|
||||
|
||||
|
|
|
@ -116,31 +116,31 @@ public class AbstractPartitionAssignorTest {
|
|||
public void testUseRackAwareAssignment() {
|
||||
AbstractPartitionAssignor assignor = new RangeAssignor();
|
||||
String[] racks = new String[] {"a", "b", "c"};
|
||||
Set<String> allRacks = Utils.mkSet(racks);
|
||||
Set<String> twoRacks = Utils.mkSet("a", "b");
|
||||
Set<String> allRacks = Set.of(racks);
|
||||
Set<String> twoRacks = Set.of("a", "b");
|
||||
Map<TopicPartition, Set<String>> partitionsOnAllRacks = new HashMap<>();
|
||||
Map<TopicPartition, Set<String>> partitionsOnSubsetOfRacks = new HashMap<>();
|
||||
for (int i = 0; i < 10; i++) {
|
||||
TopicPartition tp = new TopicPartition("topic", i);
|
||||
partitionsOnAllRacks.put(tp, allRacks);
|
||||
partitionsOnSubsetOfRacks.put(tp, Utils.mkSet(racks[i % racks.length]));
|
||||
partitionsOnSubsetOfRacks.put(tp, Set.of(racks[i % racks.length]));
|
||||
}
|
||||
assertFalse(assignor.useRackAwareAssignment(Collections.emptySet(), Collections.emptySet(), partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Collections.emptySet(), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(allRacks, Collections.emptySet(), Collections.emptyMap()));
|
||||
assertFalse(assignor.useRackAwareAssignment(Utils.mkSet("d"), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Set.of("d"), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(allRacks, allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(twoRacks, allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Utils.mkSet("a", "d"), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Set.of("a", "d"), allRacks, partitionsOnAllRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(allRacks, allRacks, partitionsOnSubsetOfRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(twoRacks, allRacks, partitionsOnSubsetOfRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(Utils.mkSet("a", "d"), allRacks, partitionsOnSubsetOfRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(Set.of("a", "d"), allRacks, partitionsOnSubsetOfRacks));
|
||||
|
||||
assignor.preferRackAwareLogic = true;
|
||||
assertFalse(assignor.useRackAwareAssignment(Collections.emptySet(), Collections.emptySet(), partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Collections.emptySet(), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(allRacks, Collections.emptySet(), Collections.emptyMap()));
|
||||
assertFalse(assignor.useRackAwareAssignment(Utils.mkSet("d"), allRacks, partitionsOnAllRacks));
|
||||
assertFalse(assignor.useRackAwareAssignment(Set.of("d"), allRacks, partitionsOnAllRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(allRacks, allRacks, partitionsOnAllRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(twoRacks, allRacks, partitionsOnAllRacks));
|
||||
assertTrue(assignor.useRackAwareAssignment(allRacks, allRacks, partitionsOnSubsetOfRacks));
|
||||
|
|
|
@ -126,7 +126,6 @@ import static org.apache.kafka.clients.consumer.CooperativeStickyAssignor.COOPER
|
|||
import static org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.DEFAULT_GENERATION;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.test.TestUtils.toSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -275,7 +274,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
assertEquals(0.0d, getMetric("assigned-partitions").metricValue());
|
||||
subscriptions.assignFromUser(Collections.singleton(t1p));
|
||||
assertEquals(1.0d, getMetric("assigned-partitions").metricValue());
|
||||
subscriptions.assignFromUser(Utils.mkSet(t1p, t2p));
|
||||
subscriptions.assignFromUser(Set.of(t1p, t2p));
|
||||
assertEquals(2.0d, getMetric("assigned-partitions").metricValue());
|
||||
}
|
||||
|
||||
|
@ -1091,7 +1090,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
assertEquals(toSet(oldSubscription), subscriptions.metadataTopics());
|
||||
|
||||
subscriptions.subscribe(toSet(newSubscription), Optional.of(rebalanceListener));
|
||||
assertEquals(Utils.mkSet(topic1, topic2), subscriptions.metadataTopics());
|
||||
assertEquals(Set.of(topic1, topic2), subscriptions.metadataTopics());
|
||||
|
||||
prepareJoinAndSyncResponse(consumerId, 2, newSubscription, newAssignment);
|
||||
coordinator.poll(time.timer(Long.MAX_VALUE));
|
||||
|
@ -1750,7 +1749,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
|
||||
@Test
|
||||
public void testPatternJoinGroupFollower() {
|
||||
final Set<String> subscription = Utils.mkSet(topic1, topic2);
|
||||
final Set<String> subscription = Set.of(topic1, topic2);
|
||||
final List<TopicPartition> owned = Collections.emptyList();
|
||||
final List<TopicPartition> assigned = Arrays.asList(t1p, t2p);
|
||||
|
||||
|
@ -2023,7 +2022,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
coordinator.poll(time.timer(Long.MAX_VALUE));
|
||||
|
||||
assertFalse(coordinator.rejoinNeededOrPending());
|
||||
assertEquals(mkSet(topic1, topic2), coordinator.subscriptionState().metadataTopics());
|
||||
assertEquals(Set.of(topic1, topic2), coordinator.subscriptionState().metadataTopics());
|
||||
|
||||
// a new partition is added to the topic2 that only consumerId2 is subscribed to
|
||||
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWith(1, singletonMap(topic2, 2)), false, time.milliseconds());
|
||||
|
@ -2089,7 +2088,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
@Test
|
||||
public void testSubscriptionChangeWithAuthorizationFailure() {
|
||||
// Subscribe to two topics of which only one is authorized and verify that metadata failure is propagated.
|
||||
subscriptions.subscribe(Utils.mkSet(topic1, topic2), Optional.of(rebalanceListener));
|
||||
subscriptions.subscribe(Set.of(topic1, topic2), Optional.of(rebalanceListener));
|
||||
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
|
||||
Collections.singletonMap(topic2, Errors.TOPIC_AUTHORIZATION_FAILED), singletonMap(topic1, 1)));
|
||||
assertThrows(TopicAuthorizationException.class, () -> coordinator.poll(time.timer(Long.MAX_VALUE)));
|
||||
|
@ -2104,7 +2103,7 @@ public abstract class ConsumerCoordinatorTest {
|
|||
|
||||
// Change subscription to include only the authorized topic. Complete rebalance and check that
|
||||
// references to topic2 have been removed from SubscriptionState.
|
||||
subscriptions.subscribe(Utils.mkSet(topic1), Optional.of(rebalanceListener));
|
||||
subscriptions.subscribe(Set.of(topic1), Optional.of(rebalanceListener));
|
||||
assertEquals(Collections.singleton(topic1), subscriptions.metadataTopics());
|
||||
client.prepareMetadataUpdate(RequestTestUtils.metadataUpdateWith("kafka-cluster", 1,
|
||||
Collections.emptyMap(), singletonMap(topic1, 1)));
|
||||
|
|
|
@ -69,7 +69,6 @@ import static org.apache.kafka.clients.consumer.internals.AsyncKafkaConsumer.inv
|
|||
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSortedSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -676,7 +675,7 @@ public class ConsumerMembershipManagerTest {
|
|||
assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state());
|
||||
verifyReconciliationNotTriggered(membershipManager);
|
||||
assertEquals(Collections.singletonMap(topic1, mkSortedSet(0)), membershipManager.currentAssignment().partitions);
|
||||
assertEquals(mkSet(topic2), membershipManager.topicsAwaitingReconciliation());
|
||||
assertEquals(Set.of(topic2), membershipManager.topicsAwaitingReconciliation());
|
||||
}
|
||||
|
||||
private Map<Uuid, SortedSet<Integer>> toTopicIdPartitionMap(final Assignment assignment) {
|
||||
|
@ -794,7 +793,7 @@ public class ConsumerMembershipManagerTest {
|
|||
|
||||
verifyReconciliationNotTriggered(membershipManager);
|
||||
assertEquals(MemberState.RECONCILING, membershipManager.state());
|
||||
assertEquals(mkSet(topicId1, topicId2), membershipManager.topicsAwaitingReconciliation());
|
||||
assertEquals(Set.of(topicId1, topicId2), membershipManager.topicsAwaitingReconciliation());
|
||||
clearInvocations(membershipManager, commitRequestManager);
|
||||
|
||||
// First reconciliation completes. Should trigger follow-up reconciliation to complete the assignment,
|
||||
|
@ -803,7 +802,7 @@ public class ConsumerMembershipManagerTest {
|
|||
commitFuture.complete(null);
|
||||
|
||||
assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state());
|
||||
assertEquals(mkSet(topicId2), membershipManager.topicsAwaitingReconciliation());
|
||||
assertEquals(Set.of(topicId2), membershipManager.topicsAwaitingReconciliation());
|
||||
|
||||
// After acknowledging the assignment, we should be back to RECONCILING, because we have not
|
||||
// yet reached the target assignment.
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.apache.kafka.common.requests.RequestTestUtils;
|
|||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.mockito.Mockito;
|
||||
|
@ -97,33 +96,33 @@ public class ConsumerMetadataTest {
|
|||
metadata.updateWithCurrentRequestVersion(response, false, time.milliseconds());
|
||||
|
||||
if (includeInternalTopics)
|
||||
assertEquals(Utils.mkSet("__matching_topic", "__consumer_offsets"), metadata.fetch().topics());
|
||||
assertEquals(Set.of("__matching_topic", "__consumer_offsets"), metadata.fetch().topics());
|
||||
else
|
||||
assertEquals(Collections.singleton("__matching_topic"), metadata.fetch().topics());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUserAssignment() {
|
||||
subscription.assignFromUser(Utils.mkSet(
|
||||
subscription.assignFromUser(Set.of(
|
||||
new TopicPartition("foo", 0),
|
||||
new TopicPartition("bar", 0),
|
||||
new TopicPartition("__consumer_offsets", 0)));
|
||||
testBasicSubscription(Utils.mkSet("foo", "bar"), Utils.mkSet("__consumer_offsets"));
|
||||
testBasicSubscription(Set.of("foo", "bar"), Set.of("__consumer_offsets"));
|
||||
|
||||
subscription.assignFromUser(Utils.mkSet(
|
||||
subscription.assignFromUser(Set.of(
|
||||
new TopicPartition("baz", 0),
|
||||
new TopicPartition("__consumer_offsets", 0)));
|
||||
testBasicSubscription(Utils.mkSet("baz"), Utils.mkSet("__consumer_offsets"));
|
||||
testBasicSubscription(Set.of("baz"), Set.of("__consumer_offsets"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNormalSubscription() {
|
||||
subscription.subscribe(Utils.mkSet("foo", "bar", "__consumer_offsets"), Optional.empty());
|
||||
subscription.groupSubscribe(Utils.mkSet("baz", "foo", "bar", "__consumer_offsets"));
|
||||
testBasicSubscription(Utils.mkSet("foo", "bar", "baz"), Utils.mkSet("__consumer_offsets"));
|
||||
subscription.subscribe(Set.of("foo", "bar", "__consumer_offsets"), Optional.empty());
|
||||
subscription.groupSubscribe(Set.of("baz", "foo", "bar", "__consumer_offsets"));
|
||||
testBasicSubscription(Set.of("foo", "bar", "baz"), Set.of("__consumer_offsets"));
|
||||
|
||||
subscription.resetGroupSubscription();
|
||||
testBasicSubscription(Utils.mkSet("foo", "bar"), Utils.mkSet("__consumer_offsets"));
|
||||
testBasicSubscription(Set.of("foo", "bar"), Set.of("__consumer_offsets"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -151,7 +150,7 @@ public class ConsumerMetadataTest {
|
|||
topicIds.forEach((topicName, topicId) -> assertEquals(topicId, metadataTopicIds.get(topicName)));
|
||||
assertFalse(metadata.updateRequested());
|
||||
|
||||
assertEquals(Utils.mkSet("foo", "bar"), new HashSet<>(metadata.fetch().topics()));
|
||||
assertEquals(Set.of("foo", "bar"), new HashSet<>(metadata.fetch().topics()));
|
||||
|
||||
metadata.clearTransientTopics();
|
||||
topicIds.remove("bar");
|
||||
|
|
|
@ -134,7 +134,6 @@ import static java.util.Collections.singletonMap;
|
|||
import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
|
||||
import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
|
||||
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.test.TestUtils.assertOptional;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -1443,7 +1442,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords;
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
// seek to tp0 and tp1 in two polls to generate 2 complete requests and responses
|
||||
|
||||
|
@ -1475,7 +1474,7 @@ public class FetchRequestManagerTest {
|
|||
public void testFetchOnCompletedFetchesForAllPausedPartitions() {
|
||||
buildFetcher();
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
// seek to tp0 and tp1 in two polls to generate 2 complete requests and responses
|
||||
|
||||
|
@ -1509,7 +1508,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords;
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
subscriptions.seek(tp0, 1);
|
||||
assertEquals(1, sendFetches());
|
||||
|
@ -1729,7 +1728,7 @@ public class FetchRequestManagerTest {
|
|||
// some fetched partitions cause Exception. This ensures that consumer won't lose record upon exception
|
||||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED);
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 1);
|
||||
subscriptions.seek(tp1, 1);
|
||||
|
||||
|
@ -1775,7 +1774,7 @@ public class FetchRequestManagerTest {
|
|||
// Ensure the removal of completed fetches that cause an Exception if and only if they contain empty records.
|
||||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED);
|
||||
assignFromUser(mkSet(tp0, tp1, tp2, tp3));
|
||||
assignFromUser(Set.of(tp0, tp1, tp2, tp3));
|
||||
|
||||
subscriptions.seek(tp0, 1);
|
||||
subscriptions.seek(tp1, 1);
|
||||
|
@ -1852,7 +1851,7 @@ public class FetchRequestManagerTest {
|
|||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), 2, IsolationLevel.READ_UNCOMMITTED);
|
||||
|
||||
assignFromUser(mkSet(tp0));
|
||||
assignFromUser(Set.of(tp0));
|
||||
subscriptions.seek(tp0, 1);
|
||||
assertEquals(1, sendFetches());
|
||||
Map<TopicIdPartition, FetchResponseData.PartitionData> partitions = new HashMap<>();
|
||||
|
@ -1865,7 +1864,7 @@ public class FetchRequestManagerTest {
|
|||
|
||||
assertEquals(2, fetchRecords().get(tp0).size());
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp0, tp1));
|
||||
subscriptions.assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1)));
|
||||
|
||||
assertEquals(1, sendFetches());
|
||||
|
@ -2089,7 +2088,7 @@ public class FetchRequestManagerTest {
|
|||
TopicPartition tp1 = new TopicPartition(topic1, 0);
|
||||
TopicPartition tp2 = new TopicPartition(topic2, 0);
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp1, tp2));
|
||||
subscriptions.assignFromUser(Set.of(tp1, tp2));
|
||||
|
||||
Map<String, Integer> partitionCounts = new HashMap<>();
|
||||
partitionCounts.put(topic1, 1);
|
||||
|
@ -2103,7 +2102,7 @@ public class FetchRequestManagerTest {
|
|||
int expectedBytes = 0;
|
||||
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> fetchPartitionData = new LinkedHashMap<>();
|
||||
|
||||
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
|
||||
for (TopicIdPartition tp : Set.of(tidp1, tidp2)) {
|
||||
subscriptions.seek(tp.topicPartition(), 0);
|
||||
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
|
||||
|
@ -2167,7 +2166,7 @@ public class FetchRequestManagerTest {
|
|||
@Test
|
||||
public void testFetchResponseMetricsWithOnePartitionError() {
|
||||
buildFetcher();
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 0);
|
||||
subscriptions.seek(tp1, 0);
|
||||
|
||||
|
@ -2210,7 +2209,7 @@ public class FetchRequestManagerTest {
|
|||
public void testFetchResponseMetricsWithOnePartitionAtTheWrongOffset() {
|
||||
buildFetcher();
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 0);
|
||||
subscriptions.seek(tp1, 0);
|
||||
|
||||
|
@ -2444,7 +2443,7 @@ public class FetchRequestManagerTest {
|
|||
for (ConsumerRecord<byte[], byte[]> consumerRecord : fetchedConsumerRecords) {
|
||||
fetchedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
|
||||
}
|
||||
assertEquals(mkSet("commit1-1", "commit1-2", "commit2-1"), fetchedKeys);
|
||||
assertEquals(Set.of("commit1-1", "commit1-2", "commit2-1"), fetchedKeys);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -131,7 +131,6 @@ import static java.util.Collections.singleton;
|
|||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.requests.FetchMetadata.INVALID_SESSION_ID;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.test.TestUtils.assertOptional;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -1450,7 +1449,7 @@ public class FetcherTest {
|
|||
|
||||
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords;
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
// seek to tp0 and tp1 in two polls to generate 2 complete requests and responses
|
||||
|
||||
|
@ -1482,7 +1481,7 @@ public class FetcherTest {
|
|||
public void testFetchOnCompletedFetchesForAllPausedPartitions() {
|
||||
buildFetcher();
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
// seek to tp0 and tp1 in two polls to generate 2 complete requests and responses
|
||||
|
||||
|
@ -1516,7 +1515,7 @@ public class FetcherTest {
|
|||
|
||||
Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> fetchedRecords;
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
|
||||
subscriptions.seek(tp0, 1);
|
||||
assertEquals(1, sendFetches());
|
||||
|
@ -1723,7 +1722,7 @@ public class FetcherTest {
|
|||
// some fetched partitions cause Exception. This ensures that consumer won't lose record upon exception
|
||||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED);
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 1);
|
||||
subscriptions.seek(tp1, 1);
|
||||
|
||||
|
@ -1769,7 +1768,7 @@ public class FetcherTest {
|
|||
// Ensure the removal of completed fetches that cause an Exception if and only if they contain empty records.
|
||||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), Integer.MAX_VALUE, IsolationLevel.READ_UNCOMMITTED);
|
||||
assignFromUser(mkSet(tp0, tp1, tp2, tp3));
|
||||
assignFromUser(Set.of(tp0, tp1, tp2, tp3));
|
||||
|
||||
subscriptions.seek(tp0, 1);
|
||||
subscriptions.seek(tp1, 1);
|
||||
|
@ -1846,7 +1845,7 @@ public class FetcherTest {
|
|||
buildFetcher(OffsetResetStrategy.NONE, new ByteArrayDeserializer(),
|
||||
new ByteArrayDeserializer(), 2, IsolationLevel.READ_UNCOMMITTED);
|
||||
|
||||
assignFromUser(mkSet(tp0));
|
||||
assignFromUser(Set.of(tp0));
|
||||
subscriptions.seek(tp0, 1);
|
||||
assertEquals(1, sendFetches());
|
||||
Map<TopicIdPartition, FetchResponseData.PartitionData> partitions = new HashMap<>();
|
||||
|
@ -1859,7 +1858,7 @@ public class FetcherTest {
|
|||
|
||||
assertEquals(2, fetchRecords().get(tp0).size());
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp0, tp1));
|
||||
subscriptions.assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seekUnvalidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1)));
|
||||
|
||||
assertEquals(1, sendFetches());
|
||||
|
@ -2083,7 +2082,7 @@ public class FetcherTest {
|
|||
TopicPartition tp1 = new TopicPartition(topic1, 0);
|
||||
TopicPartition tp2 = new TopicPartition(topic2, 0);
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp1, tp2));
|
||||
subscriptions.assignFromUser(Set.of(tp1, tp2));
|
||||
|
||||
Map<String, Integer> partitionCounts = new HashMap<>();
|
||||
partitionCounts.put(topic1, 1);
|
||||
|
@ -2097,7 +2096,7 @@ public class FetcherTest {
|
|||
int expectedBytes = 0;
|
||||
LinkedHashMap<TopicIdPartition, FetchResponseData.PartitionData> fetchPartitionData = new LinkedHashMap<>();
|
||||
|
||||
for (TopicIdPartition tp : mkSet(tidp1, tidp2)) {
|
||||
for (TopicIdPartition tp : Set.of(tidp1, tidp2)) {
|
||||
subscriptions.seek(tp.topicPartition(), 0);
|
||||
|
||||
MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), Compression.NONE,
|
||||
|
@ -2161,7 +2160,7 @@ public class FetcherTest {
|
|||
@Test
|
||||
public void testFetchResponseMetricsWithOnePartitionError() {
|
||||
buildFetcher();
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 0);
|
||||
subscriptions.seek(tp1, 0);
|
||||
|
||||
|
@ -2204,7 +2203,7 @@ public class FetcherTest {
|
|||
public void testFetchResponseMetricsWithOnePartitionAtTheWrongOffset() {
|
||||
buildFetcher();
|
||||
|
||||
assignFromUser(mkSet(tp0, tp1));
|
||||
assignFromUser(Set.of(tp0, tp1));
|
||||
subscriptions.seek(tp0, 0);
|
||||
subscriptions.seek(tp1, 0);
|
||||
|
||||
|
@ -2438,7 +2437,7 @@ public class FetcherTest {
|
|||
for (ConsumerRecord<byte[], byte[]> consumerRecord : fetchedConsumerRecords) {
|
||||
fetchedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
|
||||
}
|
||||
assertEquals(mkSet("commit1-1", "commit1-2", "commit2-1"), fetchedKeys);
|
||||
assertEquals(Set.of("commit1-1", "commit1-2", "commit2-1"), fetchedKeys);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -84,7 +84,6 @@ import static java.util.Collections.singleton;
|
|||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH;
|
||||
import static org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.UNDEFINED_EPOCH_OFFSET;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.test.TestUtils.assertOptional;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -747,7 +746,7 @@ public class OffsetFetcherTest {
|
|||
for (Errors retriableError : retriableErrors) {
|
||||
buildFetcher();
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp0, tp1));
|
||||
subscriptions.assignFromUser(Set.of(tp0, tp1));
|
||||
client.updateMetadata(initialUpdateResponse);
|
||||
|
||||
final long fetchTimestamp = 10L;
|
||||
|
@ -897,7 +896,7 @@ public class OffsetFetcherTest {
|
|||
public void testGetOffsetsForTimesWhenSomeTopicPartitionLeadersNotKnownInitially() {
|
||||
buildFetcher();
|
||||
|
||||
subscriptions.assignFromUser(mkSet(tp0, tp1));
|
||||
subscriptions.assignFromUser(Set.of(tp0, tp1));
|
||||
final String anotherTopic = "another-topic";
|
||||
final TopicPartition t2p0 = new TopicPartition(anotherTopic, 0);
|
||||
|
||||
|
@ -944,7 +943,7 @@ public class OffsetFetcherTest {
|
|||
buildFetcher();
|
||||
final String anotherTopic = "another-topic";
|
||||
final TopicPartition t2p0 = new TopicPartition(anotherTopic, 0);
|
||||
subscriptions.assignFromUser(mkSet(tp0, t2p0));
|
||||
subscriptions.assignFromUser(Set.of(tp0, t2p0));
|
||||
|
||||
client.reset();
|
||||
|
||||
|
@ -1132,7 +1131,7 @@ public class OffsetFetcherTest {
|
|||
@Test
|
||||
public void testOffsetValidationRequestGrouping() {
|
||||
buildFetcher();
|
||||
assignFromUser(mkSet(tp0, tp1, tp2, tp3));
|
||||
assignFromUser(Set.of(tp0, tp1, tp2, tp3));
|
||||
|
||||
metadata.updateWithCurrentRequestVersion(RequestTestUtils.metadataUpdateWithIds("dummy", 3,
|
||||
Collections.emptyMap(), singletonMap(topicName, 4),
|
||||
|
|
|
@ -61,7 +61,6 @@ import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER
|
|||
import static org.apache.kafka.common.requests.ShareGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSortedSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -428,7 +427,7 @@ public class ShareMembershipManagerTest {
|
|||
assertEquals(MemberState.ACKNOWLEDGING, membershipManager.state());
|
||||
verifyReconciliationNotTriggered(membershipManager);
|
||||
assertEquals(Collections.singletonMap(topic1, mkSortedSet(0)), membershipManager.currentAssignment().partitions);
|
||||
assertEquals(mkSet(topic2), membershipManager.topicsAwaitingReconciliation());
|
||||
assertEquals(Set.of(topic2), membershipManager.topicsAwaitingReconciliation());
|
||||
}
|
||||
|
||||
private Map<Uuid, SortedSet<Integer>> toTopicIdPartitionMap(final ShareGroupHeartbeatResponseData.Assignment assignment) {
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.kafka.common.TopicPartition;
|
|||
import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -123,15 +122,15 @@ public class SubscriptionStateTest {
|
|||
assertFalse(state.groupSubscribe(singleton(topic1)));
|
||||
assertEquals(singleton(topic1), state.metadataTopics());
|
||||
|
||||
assertTrue(state.groupSubscribe(Utils.mkSet(topic, topic1)));
|
||||
assertEquals(Utils.mkSet(topic, topic1), state.metadataTopics());
|
||||
assertTrue(state.groupSubscribe(Set.of(topic, topic1)));
|
||||
assertEquals(Set.of(topic, topic1), state.metadataTopics());
|
||||
|
||||
// `groupSubscribe` does not accumulate
|
||||
assertFalse(state.groupSubscribe(singleton(topic1)));
|
||||
assertEquals(singleton(topic1), state.metadataTopics());
|
||||
|
||||
state.subscribe(singleton("anotherTopic"), Optional.of(rebalanceListener));
|
||||
assertEquals(Utils.mkSet(topic1, "anotherTopic"), state.metadataTopics());
|
||||
assertEquals(Set.of(topic1, "anotherTopic"), state.metadataTopics());
|
||||
|
||||
assertFalse(state.groupSubscribe(singleton("anotherTopic")));
|
||||
assertEquals(singleton("anotherTopic"), state.metadataTopics());
|
||||
|
@ -192,7 +191,7 @@ public class SubscriptionStateTest {
|
|||
@Test
|
||||
public void verifyAssignmentId() {
|
||||
assertEquals(0, state.assignmentId());
|
||||
Set<TopicPartition> userAssignment = Utils.mkSet(tp0, tp1);
|
||||
Set<TopicPartition> userAssignment = Set.of(tp0, tp1);
|
||||
state.assignFromUser(userAssignment);
|
||||
assertEquals(1, state.assignmentId());
|
||||
assertEquals(userAssignment, state.assignedPartitions());
|
||||
|
@ -201,7 +200,7 @@ public class SubscriptionStateTest {
|
|||
assertEquals(2, state.assignmentId());
|
||||
assertEquals(Collections.emptySet(), state.assignedPartitions());
|
||||
|
||||
Set<TopicPartition> autoAssignment = Utils.mkSet(t1p0);
|
||||
Set<TopicPartition> autoAssignment = Set.of(t1p0);
|
||||
state.subscribe(singleton(topic1), Optional.of(rebalanceListener));
|
||||
assertTrue(state.checkAssignmentMatchedSubscription(autoAssignment));
|
||||
state.assignFromSubscribed(autoAssignment);
|
||||
|
@ -316,7 +315,7 @@ public class SubscriptionStateTest {
|
|||
|
||||
// New partition added to the assignment. Owned partitions should continue to be
|
||||
// fetchable, while the newly added should not be fetchable until callback completes.
|
||||
state.assignFromSubscribedAwaitingCallback(Utils.mkSet(tp0, tp1), singleton(tp1));
|
||||
state.assignFromSubscribedAwaitingCallback(Set.of(tp0, tp1), singleton(tp1));
|
||||
assertTrue(state.isFetchable(tp0));
|
||||
assertFalse(state.isFetchable(tp1));
|
||||
assertEquals(1, state.initializingPartitions().size());
|
||||
|
|
|
@ -16,8 +16,6 @@
|
|||
*/
|
||||
package org.apache.kafka.common;
|
||||
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -56,7 +54,7 @@ public class ClusterTest {
|
|||
new InetSocketAddress(ipAddress, 9002),
|
||||
new InetSocketAddress(hostName, 9002)
|
||||
));
|
||||
Set<String> expectedHosts = Utils.mkSet(ipAddress, hostName);
|
||||
Set<String> expectedHosts = Set.of(ipAddress, hostName);
|
||||
Set<String> actualHosts = new HashSet<>();
|
||||
for (Node node : cluster.nodes())
|
||||
actualHosts.add(node.host());
|
||||
|
@ -74,9 +72,9 @@ public class ClusterTest {
|
|||
new PartitionInfo(TOPIC_D, 0, NODES[1], NODES, NODES),
|
||||
new PartitionInfo(TOPIC_E, 0, NODES[0], NODES, NODES)
|
||||
);
|
||||
Set<String> unauthorizedTopics = Utils.mkSet(TOPIC_C);
|
||||
Set<String> invalidTopics = Utils.mkSet(TOPIC_D);
|
||||
Set<String> internalTopics = Utils.mkSet(TOPIC_E);
|
||||
Set<String> unauthorizedTopics = Set.of(TOPIC_C);
|
||||
Set<String> invalidTopics = Set.of(TOPIC_D);
|
||||
Set<String> internalTopics = Set.of(TOPIC_E);
|
||||
Cluster cluster = new Cluster("clusterId", asList(NODES), allPartitions, unauthorizedTopics,
|
||||
invalidTopics, internalTopics, NODES[1]);
|
||||
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.kafka.common.network;
|
||||
|
||||
import org.apache.kafka.common.config.SslConfigs;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.TestSslUtils;
|
||||
import org.apache.kafka.test.TestSslUtils.SslConfigsBuilder;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
@ -32,7 +31,7 @@ import java.util.Set;
|
|||
|
||||
public class CertStores {
|
||||
|
||||
public static final Set<String> KEYSTORE_PROPS = Utils.mkSet(
|
||||
public static final Set<String> KEYSTORE_PROPS = Set.of(
|
||||
SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_TYPE_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG,
|
||||
|
@ -40,7 +39,7 @@ public class CertStores {
|
|||
SslConfigs.SSL_KEYSTORE_KEY_CONFIG,
|
||||
SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG);
|
||||
|
||||
public static final Set<String> TRUSTSTORE_PROPS = Utils.mkSet(
|
||||
public static final Set<String> TRUSTSTORE_PROPS = Set.of(
|
||||
SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG,
|
||||
SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG,
|
||||
|
|
|
@ -786,7 +786,7 @@ public class SelectorTest {
|
|||
when(selectionKey.readyOps()).thenReturn(SelectionKey.OP_CONNECT);
|
||||
when(selectionKey.attachment()).thenReturn(kafkaChannel);
|
||||
|
||||
Set<SelectionKey> selectionKeys = Utils.mkSet(selectionKey);
|
||||
Set<SelectionKey> selectionKeys = Set.of(selectionKey);
|
||||
selector.pollSelectionKeys(selectionKeys, false, System.nanoTime());
|
||||
|
||||
assertFalse(selector.connected().contains(kafkaChannel.id()));
|
||||
|
|
|
@ -278,7 +278,6 @@ import org.apache.kafka.common.security.auth.SecurityProtocol;
|
|||
import org.apache.kafka.common.security.token.delegation.DelegationToken;
|
||||
import org.apache.kafka.common.security.token.delegation.TokenInformation;
|
||||
import org.apache.kafka.common.utils.SecurityUtils;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
|
@ -2696,7 +2695,7 @@ public class RequestResponseTest {
|
|||
.setReplicas(replicas)
|
||||
.setIsNew(false));
|
||||
|
||||
Set<Node> leaders = Utils.mkSet(
|
||||
Set<Node> leaders = Set.of(
|
||||
new Node(0, "test0", 1223),
|
||||
new Node(1, "test1", 1223)
|
||||
);
|
||||
|
|
|
@ -19,13 +19,13 @@ package org.apache.kafka.common.requests;
|
|||
import org.apache.kafka.common.message.UpdateFeaturesResponseData;
|
||||
import org.apache.kafka.common.protocol.ApiKeys;
|
||||
import org.apache.kafka.common.protocol.Errors;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
|
@ -67,7 +67,7 @@ public class UpdateFeaturesResponseTest {
|
|||
@ApiKeyVersionsSource(apiKey = ApiKeys.UPDATE_FEATURES)
|
||||
public void testSerialization(short version) {
|
||||
UpdateFeaturesResponse noErrorResponse = UpdateFeaturesResponse.parse(UpdateFeaturesResponse.createWithErrors(ApiError.NONE,
|
||||
Utils.mkSet("feature-1", "feature-2"), 0).serialize(version), version);
|
||||
Set.of("feature-1", "feature-2"), 0).serialize(version), version);
|
||||
|
||||
// Versions 1 and below still contain feature level results when the error is NONE.
|
||||
int expectedSize = version <= 1 ? 2 : 0;
|
||||
|
@ -76,7 +76,7 @@ public class UpdateFeaturesResponseTest {
|
|||
|
||||
ApiError error = new ApiError(Errors.INVALID_UPDATE_VERSION);
|
||||
UpdateFeaturesResponse errorResponse = UpdateFeaturesResponse.parse(UpdateFeaturesResponse.createWithErrors(error,
|
||||
Utils.mkSet("feature-1", "feature-2"), 0).serialize(version), version);
|
||||
Set.of("feature-1", "feature-2"), 0).serialize(version), version);
|
||||
assertEquals(error, errorResponse.topLevelError());
|
||||
assertEquals(0, errorResponse.data().results().size());
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ import org.apache.kafka.common.security.ssl.DefaultSslEngineFactory.SecurityStor
|
|||
import org.apache.kafka.common.security.ssl.mock.TestKeyManagerFactory;
|
||||
import org.apache.kafka.common.security.ssl.mock.TestProviderCreator;
|
||||
import org.apache.kafka.common.security.ssl.mock.TestTrustManagerFactory;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.test.TestSslUtils;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
||||
|
@ -47,6 +46,7 @@ import java.security.cert.X509Certificate;
|
|||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
import javax.net.ssl.SSLContext;
|
||||
import javax.net.ssl.SSLEngine;
|
||||
|
@ -81,7 +81,7 @@ public abstract class SslFactoryTest {
|
|||
//host and port are hints
|
||||
SSLEngine engine = sslFactory.createSslEngine("localhost", 0);
|
||||
assertNotNull(engine);
|
||||
assertEquals(Utils.mkSet(tlsProtocol), Utils.mkSet(engine.getEnabledProtocols()));
|
||||
assertEquals(Set.of(tlsProtocol), Set.of(engine.getEnabledProtocols()));
|
||||
assertFalse(engine.getUseClientMode());
|
||||
}
|
||||
}
|
||||
|
@ -95,7 +95,7 @@ public abstract class SslFactoryTest {
|
|||
sslFactory.configure(serverSslConfig);
|
||||
SSLEngine engine = sslFactory.createSslEngine("localhost", 0);
|
||||
assertNotNull(engine);
|
||||
assertEquals(Utils.mkSet(tlsProtocol), Utils.mkSet(engine.getEnabledProtocols()));
|
||||
assertEquals(Set.of(tlsProtocol), Set.of(engine.getEnabledProtocols()));
|
||||
assertFalse(engine.getUseClientMode());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -79,7 +79,6 @@ import static org.apache.kafka.common.utils.Utils.getHost;
|
|||
import static org.apache.kafka.common.utils.Utils.getPort;
|
||||
import static org.apache.kafka.common.utils.Utils.intersection;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.common.utils.Utils.murmur2;
|
||||
import static org.apache.kafka.common.utils.Utils.union;
|
||||
import static org.apache.kafka.common.utils.Utils.validHostPattern;
|
||||
|
@ -780,7 +779,7 @@ public class UtilsTest {
|
|||
|
||||
@Test
|
||||
public void testConvertTo32BitField() {
|
||||
Set<Byte> bytes = mkSet((byte) 0, (byte) 1, (byte) 5, (byte) 10, (byte) 31);
|
||||
Set<Byte> bytes = Set.of((byte) 0, (byte) 1, (byte) 5, (byte) 10, (byte) 31);
|
||||
int bitField = Utils.to32BitField(bytes);
|
||||
assertEquals(bytes, Utils.from32BitField(bitField));
|
||||
|
||||
|
@ -788,37 +787,37 @@ public class UtilsTest {
|
|||
bitField = Utils.to32BitField(bytes);
|
||||
assertEquals(bytes, Utils.from32BitField(bitField));
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> Utils.to32BitField(mkSet((byte) 0, (byte) 11, (byte) 32)));
|
||||
assertThrows(IllegalArgumentException.class, () -> Utils.to32BitField(Set.of((byte) 0, (byte) 11, (byte) 32)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnion() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> anotherSet = mkSet("c", "d", "e");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> anotherSet = Set.of("c", "d", "e");
|
||||
final Set<String> union = union(TreeSet::new, oneSet, anotherSet);
|
||||
|
||||
assertEquals(mkSet("a", "b", "c", "d", "e"), union);
|
||||
assertEquals(Set.of("a", "b", "c", "d", "e"), union);
|
||||
assertEquals(TreeSet.class, union.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnionOfOne() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> union = union(TreeSet::new, oneSet);
|
||||
|
||||
assertEquals(mkSet("a", "b", "c"), union);
|
||||
assertEquals(Set.of("a", "b", "c"), union);
|
||||
assertEquals(TreeSet.class, union.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnionOfMany() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> twoSet = mkSet("c", "d", "e");
|
||||
final Set<String> threeSet = mkSet("b", "c", "d");
|
||||
final Set<String> fourSet = mkSet("x", "y", "z");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> twoSet = Set.of("c", "d", "e");
|
||||
final Set<String> threeSet = Set.of("b", "c", "d");
|
||||
final Set<String> fourSet = Set.of("x", "y", "z");
|
||||
final Set<String> union = union(TreeSet::new, oneSet, twoSet, threeSet, fourSet);
|
||||
|
||||
assertEquals(mkSet("a", "b", "c", "d", "e", "x", "y", "z"), union);
|
||||
assertEquals(Set.of("a", "b", "c", "d", "e", "x", "y", "z"), union);
|
||||
assertEquals(TreeSet.class, union.getClass());
|
||||
}
|
||||
|
||||
|
@ -832,40 +831,40 @@ public class UtilsTest {
|
|||
|
||||
@Test
|
||||
public void testIntersection() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> anotherSet = mkSet("c", "d", "e");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> anotherSet = Set.of("c", "d", "e");
|
||||
final Set<String> intersection = intersection(TreeSet::new, oneSet, anotherSet);
|
||||
|
||||
assertEquals(mkSet("c"), intersection);
|
||||
assertEquals(Set.of("c"), intersection);
|
||||
assertEquals(TreeSet.class, intersection.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntersectionOfOne() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> intersection = intersection(TreeSet::new, oneSet);
|
||||
|
||||
assertEquals(mkSet("a", "b", "c"), intersection);
|
||||
assertEquals(Set.of("a", "b", "c"), intersection);
|
||||
assertEquals(TreeSet.class, intersection.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIntersectionOfMany() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> twoSet = mkSet("c", "d", "e");
|
||||
final Set<String> threeSet = mkSet("b", "c", "d");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> twoSet = Set.of("c", "d", "e");
|
||||
final Set<String> threeSet = Set.of("b", "c", "d");
|
||||
final Set<String> intersection = intersection(TreeSet::new, oneSet, twoSet, threeSet);
|
||||
|
||||
assertEquals(mkSet("c"), intersection);
|
||||
assertEquals(Set.of("c"), intersection);
|
||||
assertEquals(TreeSet.class, intersection.getClass());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDisjointIntersectionOfMany() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> twoSet = mkSet("c", "d", "e");
|
||||
final Set<String> threeSet = mkSet("b", "c", "d");
|
||||
final Set<String> fourSet = mkSet("x", "y", "z");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> twoSet = Set.of("c", "d", "e");
|
||||
final Set<String> threeSet = Set.of("b", "c", "d");
|
||||
final Set<String> fourSet = Set.of("x", "y", "z");
|
||||
final Set<String> intersection = intersection(TreeSet::new, oneSet, twoSet, threeSet, fourSet);
|
||||
|
||||
assertEquals(emptySet(), intersection);
|
||||
|
@ -874,11 +873,11 @@ public class UtilsTest {
|
|||
|
||||
@Test
|
||||
public void testDiff() {
|
||||
final Set<String> oneSet = mkSet("a", "b", "c");
|
||||
final Set<String> anotherSet = mkSet("c", "d", "e");
|
||||
final Set<String> oneSet = Set.of("a", "b", "c");
|
||||
final Set<String> anotherSet = Set.of("c", "d", "e");
|
||||
final Set<String> diff = diff(TreeSet::new, oneSet, anotherSet);
|
||||
|
||||
assertEquals(mkSet("a", "b"), diff);
|
||||
assertEquals(Set.of("a", "b"), diff);
|
||||
assertEquals(TreeSet.class, diff.getClass());
|
||||
}
|
||||
|
||||
|
|
|
@ -55,8 +55,7 @@ import java.util.EnumMap;
|
|||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Implementation of {@link Converter} and {@link HeaderConverter} that uses JSON to store schemas and objects. By
|
||||
|
@ -248,13 +247,13 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned {
|
|||
*/
|
||||
public JsonConverter(boolean enableAfterburner) {
|
||||
serializer = new JsonSerializer(
|
||||
mkSet(),
|
||||
Set.of(),
|
||||
JSON_NODE_FACTORY,
|
||||
enableAfterburner
|
||||
);
|
||||
|
||||
deserializer = new JsonDeserializer(
|
||||
mkSet(
|
||||
Set.of(
|
||||
// this ensures that the JsonDeserializer maintains full precision on
|
||||
// floating point numbers that cannot fit into float64
|
||||
DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.kafka.connect.mirror;
|
||||
|
||||
import org.apache.kafka.common.config.ConfigDef;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
|
@ -26,6 +25,7 @@ import java.util.Collections;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.connect.mirror.TestUtils.assertEqualsExceptClientId;
|
||||
import static org.apache.kafka.connect.mirror.TestUtils.makeProps;
|
||||
|
@ -122,7 +122,7 @@ public class MirrorCheckpointConfigTest {
|
|||
|
||||
configValues = MirrorCheckpointConfig.validate(makeProps(MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED, "true",
|
||||
MirrorCheckpointConfig.EMIT_OFFSET_SYNCS_ENABLED, "false"));
|
||||
assertEquals(configValues.keySet(), Utils.mkSet(MirrorCheckpointConfig.EMIT_OFFSET_SYNCS_ENABLED));
|
||||
assertEquals(configValues.keySet(), Set.of(MirrorCheckpointConfig.EMIT_OFFSET_SYNCS_ENABLED));
|
||||
|
||||
configValues = MirrorCheckpointConfig.validate(makeProps(MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED, "true",
|
||||
MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED, "true",
|
||||
|
|
|
@ -76,7 +76,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
|||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.ACTIVE;
|
||||
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.CLOSED;
|
||||
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime.CoordinatorState.FAILED;
|
||||
|
@ -1099,7 +1098,7 @@ public class CoordinatorRuntimeTest {
|
|||
// A new snapshot is created.
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
// Records have been replayed to the coordinator.
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
// Records have been written to the log.
|
||||
assertEquals(Collections.singletonList(
|
||||
records(timer.time().milliseconds(), "record1", "record2")
|
||||
|
@ -1119,7 +1118,7 @@ public class CoordinatorRuntimeTest {
|
|||
// A new snapshot is created.
|
||||
assertEquals(Arrays.asList(0L, 2L, 3L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
// Records have been replayed to the coordinator.
|
||||
assertEquals(mkSet("record1", "record2", "record3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2", "record3"), ctx.coordinator.coordinator().records());
|
||||
// Records have been written to the log.
|
||||
assertEquals(Arrays.asList(
|
||||
records(timer.time().milliseconds(), "record1", "record2"),
|
||||
|
@ -1137,7 +1136,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(3L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L, 3L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2", "record3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2", "record3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Arrays.asList(
|
||||
records(timer.time().milliseconds(), "record1", "record2"),
|
||||
records(timer.time().milliseconds(), "record3")
|
||||
|
@ -1325,7 +1324,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
|
||||
// Write #2. It should fail because the writer is configured to only
|
||||
// accept 1 write.
|
||||
|
@ -1337,7 +1336,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1651,7 +1650,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
// Records have been replayed to the coordinator. They are stored in
|
||||
// the pending set for now.
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(
|
||||
100L
|
||||
));
|
||||
// Records have been written to the log.
|
||||
|
@ -1683,7 +1682,7 @@ public class CoordinatorRuntimeTest {
|
|||
ControlRecordType expectedType;
|
||||
if (result == TransactionResult.COMMIT) {
|
||||
// They are now in the records set if committed.
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
expectedType = ControlRecordType.COMMIT;
|
||||
} else {
|
||||
// Or they are gone if aborted.
|
||||
|
@ -1813,7 +1812,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records());
|
||||
|
||||
// Complete transaction #1. It should fail.
|
||||
|
@ -1832,7 +1831,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records());
|
||||
}
|
||||
|
||||
|
@ -1898,7 +1897,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Collections.singletonList(
|
||||
transactionalRecords(100L, (short) 5, timer.time().milliseconds(), "record1", "record2")
|
||||
|
@ -1920,7 +1919,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Collections.singletonList(
|
||||
transactionalRecords(100L, (short) 5, timer.time().milliseconds(), "record1", "record2")
|
||||
|
@ -2285,14 +2284,14 @@ public class CoordinatorRuntimeTest {
|
|||
timer.advanceClock(10 + 1);
|
||||
|
||||
// Verify that the operation was executed.
|
||||
assertEquals(mkSet("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(1, ctx.timer.size());
|
||||
|
||||
// Advance time to fire timer #2,
|
||||
timer.advanceClock(10 + 1);
|
||||
|
||||
// Verify that the operation was executed.
|
||||
assertEquals(mkSet("record1", "record2", "record3", "record4"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record1", "record2", "record3", "record4"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(0, ctx.timer.size());
|
||||
}
|
||||
|
||||
|
@ -2367,7 +2366,7 @@ public class CoordinatorRuntimeTest {
|
|||
|
||||
// Verify that the correct operation was executed. Only the third
|
||||
// instance should have been executed here.
|
||||
assertEquals(mkSet("record3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(0, ctx.timer.size());
|
||||
}
|
||||
|
||||
|
@ -3584,7 +3583,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Collections.singletonList(0L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(mkSet("record#1"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record#1"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Collections.emptyList(), writer.entries(TP));
|
||||
|
||||
// Transactional write #2 with one record. This will flush the current batch.
|
||||
|
@ -3607,8 +3606,8 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record#2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(mkSet("record#1"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record#2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record#1"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Arrays.asList(
|
||||
records(timer.time().milliseconds(), "record#1"),
|
||||
transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2")
|
||||
|
@ -3626,8 +3625,8 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(2L, ctx.coordinator.lastWrittenOffset());
|
||||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 1L, 2L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(mkSet("record#2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(mkSet("record#1", "record#3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record#2"), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(Set.of("record#1", "record#3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Arrays.asList(
|
||||
records(timer.time().milliseconds(), "record#1"),
|
||||
transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2")
|
||||
|
@ -3652,7 +3651,7 @@ public class CoordinatorRuntimeTest {
|
|||
assertEquals(0L, ctx.coordinator.lastCommittedOffset());
|
||||
assertEquals(Arrays.asList(0L, 1L, 2L, 3L, 4L), ctx.coordinator.snapshotRegistry().epochsList());
|
||||
assertEquals(Collections.emptySet(), ctx.coordinator.coordinator().pendingRecords(100L));
|
||||
assertEquals(mkSet("record#1", "record#2", "record#3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Set.of("record#1", "record#2", "record#3"), ctx.coordinator.coordinator().records());
|
||||
assertEquals(Arrays.asList(
|
||||
records(timer.time().milliseconds(), "record#1"),
|
||||
transactionalRecords(100L, (short) 50, timer.time().milliseconds(), "record#2"),
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.kafka.common.record.MemoryRecords
|
|||
import org.apache.kafka.common.record.SimpleRecord
|
||||
import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
|
||||
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, FetchMetadata => JFetchMetadata}
|
||||
import org.apache.kafka.common.utils.Utils
|
||||
import org.apache.kafka.server.util.MockTime
|
||||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api.{AfterEach, Test, Timeout}
|
||||
|
@ -1739,7 +1738,7 @@ class FetchSessionTest {
|
|||
val resp1 = context1.updateAndGenerateResponseData(respData)
|
||||
assertEquals(Errors.NONE, resp1.error)
|
||||
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
|
||||
assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp1.responseData(topicNames, request1.version).keySet)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp1.responseData(topicNames, request1.version).keySet)
|
||||
|
||||
// Incremental fetch context returns partitions with divergent epoch even if none
|
||||
// of the other conditions for return are met.
|
||||
|
@ -1768,7 +1767,7 @@ class FetchSessionTest {
|
|||
val resp3 = context2.updateAndGenerateResponseData(respData)
|
||||
assertEquals(Errors.NONE, resp3.error)
|
||||
assertEquals(resp1.sessionId, resp3.sessionId)
|
||||
assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, request2.version).keySet)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, request2.version).keySet)
|
||||
|
||||
// Partitions that meet other conditions should be returned regardless of whether
|
||||
// divergingEpoch is set or not.
|
||||
|
@ -1780,7 +1779,7 @@ class FetchSessionTest {
|
|||
val resp4 = context2.updateAndGenerateResponseData(respData)
|
||||
assertEquals(Errors.NONE, resp4.error)
|
||||
assertEquals(resp1.sessionId, resp4.sessionId)
|
||||
assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp4.responseData(topicNames, request2.version).keySet)
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1824,7 +1823,7 @@ class FetchSessionTest {
|
|||
val resp1 = context1.updateAndGenerateResponseData(respData1)
|
||||
assertEquals(Errors.NONE, resp1.error)
|
||||
assertNotEquals(INVALID_SESSION_ID, resp1.sessionId)
|
||||
assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition, tp3.topicPartition), resp1.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet())
|
||||
|
||||
// Incremental fetch context returns partitions with changes but only deprioritizes
|
||||
// the partitions with records
|
||||
|
@ -1864,7 +1863,7 @@ class FetchSessionTest {
|
|||
val resp3 = context2.updateAndGenerateResponseData(respData3)
|
||||
assertEquals(Errors.NONE, resp3.error)
|
||||
assertEquals(resp1.sessionId, resp3.sessionId)
|
||||
assertEquals(Utils.mkSet(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
|
||||
assertEquals(util.Set.of(tp1.topicPartition, tp2.topicPartition), resp3.responseData(topicNames, ApiKeys.FETCH.latestVersion()).keySet)
|
||||
|
||||
// Only the partitions whose returned records in the last response
|
||||
// were deprioritized
|
||||
|
|
|
@ -72,8 +72,8 @@ import java.util.ArrayList;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.common.runtime.TestUtil.requestContext;
|
||||
import static org.apache.kafka.coordinator.group.GroupCoordinatorShard.CLASSIC_GROUP_SIZE_COUNTER_KEY;
|
||||
import static org.apache.kafka.coordinator.group.GroupCoordinatorShard.DEFAULT_GROUP_GAUGES_UPDATE_INTERVAL_MS;
|
||||
|
@ -973,7 +973,7 @@ public class GroupCoordinatorShardTest {
|
|||
@SuppressWarnings("unchecked")
|
||||
ArgumentCaptor<List<CoordinatorRecord>> recordsCapture = ArgumentCaptor.forClass(List.class);
|
||||
|
||||
when(groupMetadataManager.groupIds()).thenReturn(mkSet("group-id", "other-group-id"));
|
||||
when(groupMetadataManager.groupIds()).thenReturn(Set.of("group-id", "other-group-id"));
|
||||
when(offsetMetadataManager.cleanupExpiredOffsets(eq("group-id"), recordsCapture.capture()))
|
||||
.thenAnswer(invocation -> {
|
||||
List<CoordinatorRecord> records = recordsCapture.getValue();
|
||||
|
|
|
@ -117,7 +117,6 @@ import static org.apache.kafka.common.protocol.Errors.NOT_COORDINATOR;
|
|||
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH;
|
||||
import static org.apache.kafka.common.requests.ConsumerGroupHeartbeatRequest.LEAVE_GROUP_STATIC_MEMBER_EPOCH;
|
||||
import static org.apache.kafka.common.requests.JoinGroupRequest.UNKNOWN_MEMBER_ID;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.Assertions.assertRecordEquals;
|
||||
import static org.apache.kafka.coordinator.group.Assertions.assertRecordsEquals;
|
||||
import static org.apache.kafka.coordinator.group.Assertions.assertResponseEquals;
|
||||
|
@ -2825,8 +2824,8 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Arrays.asList("foo", "bar"))
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M1 in group 2 subscribes to foo, bar and zar.
|
||||
|
@ -2835,9 +2834,9 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Arrays.asList("foo", "bar", "zar"))
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M2 in group 1 subscribes to bar and zar.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId1,
|
||||
|
@ -2845,9 +2844,9 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Arrays.asList("bar", "zar"))
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M2 in group 2 subscribes to foo and bar.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2,
|
||||
|
@ -2855,17 +2854,17 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Arrays.asList("foo", "bar"))
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M1 in group 1 is removed.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupCurrentAssignmentTombstoneRecord(groupId1, "group1-m1"));
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId1, "group1-m1"));
|
||||
|
||||
assertEquals(mkSet(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M1 in group 2 subscribes to nothing.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2,
|
||||
|
@ -2873,9 +2872,9 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Collections.emptyList())
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1, groupId2), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M2 in group 2 subscribes to foo.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2,
|
||||
|
@ -2883,9 +2882,9 @@ public class GroupMetadataManagerTest {
|
|||
.setSubscribedTopicNames(Collections.singletonList("foo"))
|
||||
.build()));
|
||||
|
||||
assertEquals(mkSet(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId2), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M2 in group 2 subscribes to nothing.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId2,
|
||||
|
@ -2894,8 +2893,8 @@ public class GroupMetadataManagerTest {
|
|||
.build()));
|
||||
|
||||
assertEquals(Collections.emptySet(), context.groupMetadataManager.groupsSubscribedToTopic("foo"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(mkSet(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("bar"));
|
||||
assertEquals(Set.of(groupId1), context.groupMetadataManager.groupsSubscribedToTopic("zar"));
|
||||
|
||||
// M2 in group 1 subscribes to nothing.
|
||||
context.replay(GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionRecord(groupId1,
|
||||
|
@ -5708,7 +5707,7 @@ public class GroupMetadataManagerTest {
|
|||
leaderJoinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id")
|
||||
Set.of("leader-instance-id", "follower-instance-id")
|
||||
);
|
||||
|
||||
assertEquals(rebalanceResult.leaderId, leaderJoinResult.joinFuture.get().memberId());
|
||||
|
@ -5838,7 +5837,7 @@ public class GroupMetadataManagerTest {
|
|||
leaderJoinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id")
|
||||
Set.of("leader-instance-id", "follower-instance-id")
|
||||
);
|
||||
|
||||
assertTrue(duplicateFollowerJoinResult.joinFuture.isDone());
|
||||
|
@ -5984,7 +5983,7 @@ public class GroupMetadataManagerTest {
|
|||
joinResult.joinFuture.get(),
|
||||
group,
|
||||
STABLE,
|
||||
supportSkippingAssignment ? mkSet("leader-instance-id", "follower-instance-id") : Collections.emptySet()
|
||||
supportSkippingAssignment ? Set.of("leader-instance-id", "follower-instance-id") : Collections.emptySet()
|
||||
);
|
||||
|
||||
GroupMetadataManagerTestContext.JoinResult oldLeaderJoinResult = context.sendClassicGroupJoin(
|
||||
|
@ -6174,7 +6173,7 @@ public class GroupMetadataManagerTest {
|
|||
joinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id")
|
||||
Set.of("leader-instance-id", "follower-instance-id")
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -6236,7 +6235,7 @@ public class GroupMetadataManagerTest {
|
|||
joinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id")
|
||||
Set.of("leader-instance-id", "follower-instance-id")
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -6497,7 +6496,7 @@ public class GroupMetadataManagerTest {
|
|||
leaderJoinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id")
|
||||
Set.of("leader-instance-id", "follower-instance-id")
|
||||
);
|
||||
|
||||
JoinGroupResponseData expectedFollowerResponse = new JoinGroupResponseData()
|
||||
|
@ -6939,7 +6938,7 @@ public class GroupMetadataManagerTest {
|
|||
leaderJoinResult.joinFuture.get(),
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id", newMemberInstanceId)
|
||||
Set.of("leader-instance-id", "follower-instance-id", newMemberInstanceId)
|
||||
);
|
||||
|
||||
JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData()
|
||||
|
@ -7035,7 +7034,7 @@ public class GroupMetadataManagerTest {
|
|||
newLeaderResponse,
|
||||
group,
|
||||
COMPLETING_REBALANCE,
|
||||
mkSet("leader-instance-id", "follower-instance-id", newMemberInstanceId)
|
||||
Set.of("leader-instance-id", "follower-instance-id", newMemberInstanceId)
|
||||
);
|
||||
|
||||
JoinGroupResponseData expectedNewMemberResponse = new JoinGroupResponseData()
|
||||
|
@ -8902,15 +8901,15 @@ public class GroupMetadataManagerTest {
|
|||
assertTrue(group.isInState(COMPLETING_REBALANCE));
|
||||
|
||||
assertEquals(
|
||||
mkSet(rebalanceResult.leaderId, rebalanceResult.followerId, joinResult.joinFuture.get().memberId()),
|
||||
Set.of(rebalanceResult.leaderId, rebalanceResult.followerId, joinResult.joinFuture.get().memberId()),
|
||||
group.allMemberIds()
|
||||
);
|
||||
assertEquals(
|
||||
mkSet(rebalanceResult.leaderId, rebalanceResult.followerId),
|
||||
Set.of(rebalanceResult.leaderId, rebalanceResult.followerId),
|
||||
group.allStaticMemberIds()
|
||||
);
|
||||
assertEquals(
|
||||
mkSet(joinResult.joinFuture.get().memberId()),
|
||||
Set.of(joinResult.joinFuture.get().memberId()),
|
||||
group.allDynamicMemberIds()
|
||||
);
|
||||
|
||||
|
|
|
@ -31,7 +31,6 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -56,7 +55,7 @@ public class GroupSpecImplTest {
|
|||
members.put(TEST_MEMBER, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topicId),
|
||||
Set.of(topicId),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -99,12 +98,12 @@ public class GroupSpecImplTest {
|
|||
Map<Uuid, Set<Integer>> topicPartitions = new HashMap<>();
|
||||
topicPartitions.put(
|
||||
topicId,
|
||||
mkSet(0, 1)
|
||||
Set.of(0, 1)
|
||||
);
|
||||
members.put(TEST_MEMBER, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topicId),
|
||||
Set.of(topicId),
|
||||
new Assignment(topicPartitions)
|
||||
));
|
||||
|
||||
|
|
|
@ -38,7 +38,6 @@ import java.util.Optional;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.assertAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
|
@ -150,14 +149,14 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -302,7 +301,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1),
|
||||
mkTopicAssignment(topic2Uuid, 0, 1)
|
||||
|
@ -312,7 +311,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 2),
|
||||
mkTopicAssignment(topic2Uuid, 2)
|
||||
|
@ -365,7 +364,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 2),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -375,7 +374,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 1, 2)
|
||||
|
@ -427,7 +426,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 2),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -437,7 +436,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkOrderedAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 1, 2)
|
||||
|
@ -448,7 +447,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -499,7 +498,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -509,7 +508,7 @@ public class OptimizedUniformAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 1)
|
||||
|
|
|
@ -39,7 +39,6 @@ import java.util.Optional;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
|
@ -120,7 +119,7 @@ public class RangeAssignorTest {
|
|||
new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic2Uuid),
|
||||
Set.of(topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
)
|
||||
);
|
||||
|
@ -154,14 +153,14 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -214,21 +213,21 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic3Uuid),
|
||||
Set.of(topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic2Uuid, topic3Uuid),
|
||||
Set.of(topic2Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -279,21 +278,21 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -499,7 +498,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -509,7 +508,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 1)
|
||||
|
@ -520,7 +519,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -571,7 +570,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1),
|
||||
mkTopicAssignment(topic2Uuid, 0, 1)
|
||||
|
@ -581,7 +580,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 2),
|
||||
mkTopicAssignment(topic2Uuid, 2)
|
||||
|
@ -632,7 +631,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1),
|
||||
mkTopicAssignment(topic2Uuid, 0, 1)
|
||||
|
@ -642,7 +641,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 2),
|
||||
mkTopicAssignment(topic2Uuid, 2)
|
||||
|
@ -653,7 +652,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -706,7 +705,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1),
|
||||
mkTopicAssignment(topic2Uuid, 0, 1)
|
||||
|
@ -716,7 +715,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 2),
|
||||
mkTopicAssignment(topic2Uuid, 2)
|
||||
|
@ -727,7 +726,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid),
|
||||
Set.of(topic1Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -780,7 +779,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 2),
|
||||
mkTopicAssignment(topic2Uuid, 2)
|
||||
|
@ -834,7 +833,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid),
|
||||
Set.of(topic1Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1, 2),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -844,7 +843,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid, topic3Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic2Uuid, 1)
|
||||
))
|
||||
|
@ -853,7 +852,7 @@ public class RangeAssignorTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic2Uuid),
|
||||
Set.of(topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic2Uuid, 2),
|
||||
mkTopicAssignment(topic3Uuid, 0, 1)
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Iterator;
|
|||
import java.util.NoSuchElementException;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -95,8 +94,8 @@ public class RangeSetTest {
|
|||
@Test
|
||||
void testContainsAll() {
|
||||
RangeSet rangeSet = new RangeSet(5, 10);
|
||||
assertTrue(rangeSet.containsAll(mkSet(5, 6, 7, 8, 9)));
|
||||
assertFalse(rangeSet.containsAll(mkSet(5, 6, 10)));
|
||||
assertTrue(rangeSet.containsAll(Set.of(5, 6, 7, 8, 9)));
|
||||
assertFalse(rangeSet.containsAll(Set.of(5, 6, 10)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -110,8 +109,8 @@ public class RangeSetTest {
|
|||
RangeSet rangeSet1 = new RangeSet(5, 10);
|
||||
RangeSet rangeSet2 = new RangeSet(5, 10);
|
||||
RangeSet rangeSet3 = new RangeSet(6, 10);
|
||||
Set<Integer> set = mkSet(5, 6, 7, 8, 9);
|
||||
HashSet<Integer> hashSet = new HashSet<>(mkSet(6, 7, 8, 9));
|
||||
Set<Integer> set = Set.of(5, 6, 7, 8, 9);
|
||||
HashSet<Integer> hashSet = new HashSet<>(Set.of(6, 7, 8, 9));
|
||||
|
||||
assertEquals(rangeSet1, rangeSet2);
|
||||
assertNotEquals(rangeSet1, rangeSet3);
|
||||
|
|
|
@ -35,7 +35,6 @@ import java.util.Optional;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HETEROGENEOUS;
|
||||
|
@ -135,7 +134,7 @@ public class SimpleAssignorTest {
|
|||
new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_2_UUID),
|
||||
Set.of(TOPIC_2_UUID),
|
||||
Assignment.EMPTY
|
||||
)
|
||||
);
|
||||
|
@ -169,14 +168,14 @@ public class SimpleAssignorTest {
|
|||
members.put(MEMBER_A, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_1_UUID, TOPIC_3_UUID),
|
||||
Set.of(TOPIC_1_UUID, TOPIC_3_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(MEMBER_B, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_1_UUID, TOPIC_3_UUID),
|
||||
Set.of(TOPIC_1_UUID, TOPIC_3_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -229,14 +228,14 @@ public class SimpleAssignorTest {
|
|||
members.put(MEMBER_A, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_1_UUID, TOPIC_2_UUID),
|
||||
Set.of(TOPIC_1_UUID, TOPIC_2_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
members.put(MEMBER_B, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_3_UUID),
|
||||
Set.of(TOPIC_3_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -244,7 +243,7 @@ public class SimpleAssignorTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_2_UUID, TOPIC_3_UUID),
|
||||
Set.of(TOPIC_2_UUID, TOPIC_3_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -295,7 +294,7 @@ public class SimpleAssignorTest {
|
|||
members.put(MEMBER_A, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(TOPIC_1_UUID, TOPIC_2_UUID),
|
||||
Set.of(TOPIC_1_UUID, TOPIC_2_UUID),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
|
|
@ -35,7 +35,6 @@ import java.util.Optional;
|
|||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.assertAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.invertedTargetAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
|
@ -156,7 +155,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -288,7 +287,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 3),
|
||||
mkTopicAssignment(topic2Uuid, 0)
|
||||
|
@ -298,7 +297,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid, topic3Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 4, 5),
|
||||
mkTopicAssignment(topic2Uuid, 1, 2, 3),
|
||||
|
@ -363,7 +362,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1, 2, 3),
|
||||
mkTopicAssignment(topic3Uuid, 0, 1)
|
||||
|
@ -373,7 +372,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid, topic3Uuid, topic4Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid, topic3Uuid, topic4Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic2Uuid, 0, 1, 2),
|
||||
mkTopicAssignment(topic4Uuid, 0, 1, 2)
|
||||
|
@ -435,7 +434,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 1, 2)
|
||||
|
@ -446,7 +445,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberC, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
@ -501,7 +500,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic3Uuid),
|
||||
Set.of(topic1Uuid, topic3Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 0, 1, 2),
|
||||
mkTopicAssignment(topic3Uuid, 0, 1)
|
||||
|
@ -573,7 +572,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberB, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid, topic2Uuid),
|
||||
Set.of(topic1Uuid, topic2Uuid),
|
||||
new Assignment(mkAssignment(
|
||||
mkTopicAssignment(topic1Uuid, 1),
|
||||
mkTopicAssignment(topic2Uuid, 0, 2, 4)
|
||||
|
@ -617,7 +616,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
|
|||
members.put(memberA, new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.empty(),
|
||||
Optional.empty(),
|
||||
mkSet(topic1Uuid),
|
||||
Set.of(topic1Uuid),
|
||||
Assignment.EMPTY
|
||||
));
|
||||
|
||||
|
|
|
@ -68,7 +68,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
import static org.apache.kafka.coordinator.group.classic.ClassicGroupState.COMPLETING_REBALANCE;
|
||||
|
@ -384,14 +383,14 @@ public class ClassicGroupTest {
|
|||
);
|
||||
|
||||
// by default, the group supports everything
|
||||
assertTrue(group.supportsProtocols(protocolType, mkSet("range", "roundrobin")));
|
||||
assertTrue(group.supportsProtocols(protocolType, Set.of("range", "roundrobin")));
|
||||
|
||||
group.add(member1);
|
||||
group.transitionTo(PREPARING_REBALANCE);
|
||||
|
||||
assertTrue(group.supportsProtocols(protocolType, mkSet("roundrobin", "foo")));
|
||||
assertTrue(group.supportsProtocols(protocolType, mkSet("range", "bar")));
|
||||
assertFalse(group.supportsProtocols(protocolType, mkSet("foo", "bar")));
|
||||
assertTrue(group.supportsProtocols(protocolType, Set.of("roundrobin", "foo")));
|
||||
assertTrue(group.supportsProtocols(protocolType, Set.of("range", "bar")));
|
||||
assertFalse(group.supportsProtocols(protocolType, Set.of("foo", "bar")));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -37,7 +37,6 @@ import java.util.Map;
|
|||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.Assertions.assertUnorderedListEquals;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
|
@ -286,7 +285,7 @@ public class TargetAssignmentBuilderTest {
|
|||
assertEquals(new MemberSubscriptionAndAssignmentImpl(
|
||||
Optional.of("rackId"),
|
||||
Optional.of("instanceId"),
|
||||
new TopicIds(mkSet("bar", "foo", "zar"), topicsImage),
|
||||
new TopicIds(Set.of("bar", "foo", "zar"), topicsImage),
|
||||
assignment
|
||||
), subscriptionSpec);
|
||||
}
|
||||
|
|
|
@ -26,7 +26,6 @@ import java.util.Collections;
|
|||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
@ -51,7 +50,7 @@ public class TopicIdsTest {
|
|||
|
||||
@Test
|
||||
public void testSize() {
|
||||
Set<String> topicNames = mkSet("foo", "bar", "baz");
|
||||
Set<String> topicNames = Set.of("foo", "bar", "baz");
|
||||
Set<Uuid> topicIds = new TopicIds(topicNames, TopicsImage.EMPTY);
|
||||
assertEquals(topicNames.size(), topicIds.size());
|
||||
}
|
||||
|
@ -76,7 +75,7 @@ public class TopicIdsTest {
|
|||
.build()
|
||||
.topics();
|
||||
|
||||
Set<Uuid> topicIds = new TopicIds(mkSet("foo", "bar", "baz"), topicsImage);
|
||||
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz"), topicsImage);
|
||||
|
||||
assertTrue(topicIds.contains(fooUuid));
|
||||
assertTrue(topicIds.contains(barUuid));
|
||||
|
@ -98,13 +97,13 @@ public class TopicIdsTest {
|
|||
.build()
|
||||
.topics();
|
||||
|
||||
Set<Uuid> topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage);
|
||||
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
|
||||
|
||||
assertTrue(topicIds.contains(fooUuid));
|
||||
assertTrue(topicIds.contains(barUuid));
|
||||
assertTrue(topicIds.contains(bazUuid));
|
||||
assertTrue(topicIds.contains(quxUuid));
|
||||
assertTrue(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid, quxUuid)));
|
||||
assertTrue(topicIds.containsAll(Set.of(fooUuid, barUuid, bazUuid, quxUuid)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -121,13 +120,13 @@ public class TopicIdsTest {
|
|||
.build()
|
||||
.topics();
|
||||
|
||||
Set<Uuid> topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage);
|
||||
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
|
||||
|
||||
assertTrue(topicIds.contains(fooUuid));
|
||||
assertTrue(topicIds.contains(barUuid));
|
||||
assertTrue(topicIds.contains(bazUuid));
|
||||
assertTrue(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid)));
|
||||
assertFalse(topicIds.containsAll(mkSet(fooUuid, barUuid, bazUuid, quxUuid)));
|
||||
assertTrue(topicIds.containsAll(Set.of(fooUuid, barUuid, bazUuid)));
|
||||
assertFalse(topicIds.containsAll(Set.of(fooUuid, barUuid, bazUuid, quxUuid)));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -144,8 +143,8 @@ public class TopicIdsTest {
|
|||
.build()
|
||||
.topics();
|
||||
|
||||
Set<Uuid> topicIds = new TopicIds(mkSet("foo", "bar", "baz", "qux"), topicsImage);
|
||||
Set<Uuid> expectedIds = mkSet(fooUuid, barUuid, bazUuid, quxUuid);
|
||||
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
|
||||
Set<Uuid> expectedIds = Set.of(fooUuid, barUuid, bazUuid, quxUuid);
|
||||
Set<Uuid> actualIds = new HashSet<>(topicIds);
|
||||
|
||||
assertEquals(expectedIds, actualIds);
|
||||
|
@ -167,8 +166,8 @@ public class TopicIdsTest {
|
|||
.build()
|
||||
.topics();
|
||||
|
||||
Set<Uuid> topicIds = new TopicIds(mkSet("foo", "bar", "baz", "quux"), topicsImage);
|
||||
Set<Uuid> expectedIds = mkSet(fooUuid, barUuid, bazUuid);
|
||||
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "quux"), topicsImage);
|
||||
Set<Uuid> expectedIds = Set.of(fooUuid, barUuid, bazUuid);
|
||||
Set<Uuid> actualIds = new HashSet<>(topicIds);
|
||||
|
||||
assertEquals(expectedIds, actualIds);
|
||||
|
|
|
@ -39,7 +39,6 @@ import java.util.OptionalInt;
|
|||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
import static org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember.classicProtocolListFromJoinRequestProtocolCollection;
|
||||
|
@ -78,7 +77,7 @@ public class ConsumerGroupMemberTest {
|
|||
assertEquals("rack-id", member.rackId());
|
||||
assertEquals("client-id", member.clientId());
|
||||
assertEquals("hostname", member.clientHost());
|
||||
assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals(Set.of("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals("regex", member.subscribedTopicRegex());
|
||||
assertEquals("range", member.serverAssignorName().get());
|
||||
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions());
|
||||
|
@ -185,7 +184,7 @@ public class ConsumerGroupMemberTest {
|
|||
assertEquals("new-instance-id", updatedMember.instanceId());
|
||||
assertEquals("new-rack-id", updatedMember.rackId());
|
||||
// Names are sorted.
|
||||
assertEquals(mkSet("zar"), updatedMember.subscribedTopicNames());
|
||||
assertEquals(Set.of("zar"), updatedMember.subscribedTopicNames());
|
||||
assertEquals("new-regex", updatedMember.subscribedTopicRegex());
|
||||
assertEquals("new-assignor", updatedMember.serverAssignorName().get());
|
||||
}
|
||||
|
@ -212,7 +211,7 @@ public class ConsumerGroupMemberTest {
|
|||
assertEquals("rack-id", member.rackId());
|
||||
assertEquals("client-id", member.clientId());
|
||||
assertEquals("host-id", member.clientHost());
|
||||
assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals(Set.of("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals("regex", member.subscribedTopicRegex());
|
||||
assertEquals("range", member.serverAssignorName().get());
|
||||
assertEquals(
|
||||
|
|
|
@ -28,8 +28,8 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
|
||||
import static org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -59,7 +59,7 @@ public class ShareGroupMemberTest {
|
|||
assertEquals("rack-id", member.rackId());
|
||||
assertEquals("client-id", member.clientId());
|
||||
assertEquals("hostname", member.clientHost());
|
||||
assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals(Set.of("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals(mkAssignment(mkTopicAssignment(topicId1, 1, 2, 3)), member.assignedPartitions());
|
||||
}
|
||||
|
||||
|
@ -123,7 +123,7 @@ public class ShareGroupMemberTest {
|
|||
assertNull(member.instanceId());
|
||||
assertEquals("new-rack-id", updatedMember.rackId());
|
||||
// Names are sorted.
|
||||
assertEquals(mkSet("zar"), updatedMember.subscribedTopicNames());
|
||||
assertEquals(Set.of("zar"), updatedMember.subscribedTopicNames());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -142,7 +142,7 @@ public class ShareGroupMemberTest {
|
|||
assertEquals("rack-id", member.rackId());
|
||||
assertEquals("client-id", member.clientId());
|
||||
assertEquals("host-id", member.clientHost());
|
||||
assertEquals(mkSet("bar", "foo"), member.subscribedTopicNames());
|
||||
assertEquals(Set.of("bar", "foo"), member.subscribedTopicNames());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -29,7 +29,6 @@ import org.apache.kafka.common.resource.ResourcePattern;
|
|||
import org.apache.kafka.common.security.auth.KafkaPrincipal;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.SecurityUtils;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.authorizer.Action;
|
||||
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
|
||||
import org.apache.kafka.server.authorizer.AuthorizationResult;
|
||||
|
@ -470,7 +469,7 @@ public class StandardAuthorizerData {
|
|||
KafkaPrincipal basePrincipal = sessionPrincipal.getClass().equals(KafkaPrincipal.class)
|
||||
? sessionPrincipal
|
||||
: new KafkaPrincipal(sessionPrincipal.getPrincipalType(), sessionPrincipal.getName());
|
||||
return Utils.mkSet(basePrincipal, WILDCARD_KAFKA_PRINCIPAL);
|
||||
return Set.of(basePrincipal, WILDCARD_KAFKA_PRINCIPAL);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -483,7 +483,7 @@ public class ReplicationControlManagerTest {
|
|||
}
|
||||
|
||||
void fenceBrokers(Integer... brokerIds) {
|
||||
fenceBrokers(Utils.mkSet(brokerIds));
|
||||
fenceBrokers(Set.of(brokerIds));
|
||||
}
|
||||
|
||||
void fenceBrokers(Set<Integer> brokerIds) {
|
||||
|
@ -1022,13 +1022,13 @@ public class ReplicationControlManagerTest {
|
|||
assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition));
|
||||
ctx.alterTopicConfig("foo", TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3");
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(2, 3));
|
||||
ctx.fenceBrokers(Set.of(2, 3));
|
||||
|
||||
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
|
||||
assertArrayEquals(new int[]{3}, partition.elr, partition.toString());
|
||||
assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
|
||||
ctx.fenceBrokers(Set.of(1, 2, 3));
|
||||
|
||||
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
|
||||
assertArrayEquals(new int[]{1, 3}, partition.elr, partition.toString());
|
||||
|
@ -1106,14 +1106,14 @@ public class ReplicationControlManagerTest {
|
|||
assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition));
|
||||
ctx.alterTopicConfig("foo", TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3");
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
|
||||
ctx.fenceBrokers(Set.of(1, 2, 3));
|
||||
|
||||
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
|
||||
assertArrayEquals(new int[]{2, 3}, partition.elr, partition.toString());
|
||||
assertArrayEquals(new int[]{}, partition.lastKnownElr, partition.toString());
|
||||
|
||||
ctx.unfenceBrokers(2);
|
||||
ctx.fenceBrokers(Utils.mkSet(0, 1));
|
||||
ctx.fenceBrokers(Set.of(0, 1));
|
||||
partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
|
||||
assertArrayEquals(new int[]{0, 3}, partition.elr, partition.toString());
|
||||
assertArrayEquals(new int[]{2}, partition.isr, partition.toString());
|
||||
|
@ -1136,7 +1136,7 @@ public class ReplicationControlManagerTest {
|
|||
assertEquals(OptionalInt.of(0), ctx.currentLeader(topicIdPartition));
|
||||
ctx.alterTopicConfig("foo", TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3");
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
|
||||
ctx.fenceBrokers(Set.of(1, 2, 3));
|
||||
|
||||
PartitionRegistration partition = replicationControl.getPartition(topicIdPartition.topicId(), topicIdPartition.partitionId());
|
||||
assertArrayEquals(new int[]{2, 3}, partition.elr, partition.toString());
|
||||
|
@ -2270,8 +2270,8 @@ public class ReplicationControlManagerTest {
|
|||
TopicIdPartition partition1 = new TopicIdPartition(fooId, 1);
|
||||
TopicIdPartition partition2 = new TopicIdPartition(fooId, 2);
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(2, 3));
|
||||
ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
|
||||
ctx.fenceBrokers(Set.of(2, 3));
|
||||
ctx.fenceBrokers(Set.of(1, 2, 3));
|
||||
|
||||
assertLeaderAndIsr(replication, partition0, NO_LEADER, new int[]{1});
|
||||
assertLeaderAndIsr(replication, partition1, 4, new int[]{4});
|
||||
|
@ -2352,8 +2352,8 @@ public class ReplicationControlManagerTest {
|
|||
Uuid fooId = ctx.createTestTopic("foo", new int[][]{new int[]{1, 2, 3}}).topicId();
|
||||
TopicIdPartition partition = new TopicIdPartition(fooId, 0);
|
||||
|
||||
ctx.fenceBrokers(Utils.mkSet(2, 3));
|
||||
ctx.fenceBrokers(Utils.mkSet(1, 2, 3));
|
||||
ctx.fenceBrokers(Set.of(2, 3));
|
||||
ctx.fenceBrokers(Set.of(1, 2, 3));
|
||||
ctx.unfenceBrokers(2);
|
||||
|
||||
assertLeaderAndIsr(replication, partition, NO_LEADER, new int[]{1});
|
||||
|
@ -2406,7 +2406,7 @@ public class ReplicationControlManagerTest {
|
|||
new int[]{1, 2, 3}, new int[]{2, 3, 4}, new int[]{0, 2, 1}}).topicId();
|
||||
|
||||
assertTrue(ctx.clusterControl.fencedBrokerIds().isEmpty());
|
||||
ctx.fenceBrokers(Utils.mkSet(2, 3));
|
||||
ctx.fenceBrokers(Set.of(2, 3));
|
||||
|
||||
PartitionRegistration partition0 = replication.getPartition(fooId, 0);
|
||||
PartitionRegistration partition1 = replication.getPartition(fooId, 1);
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.raft;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -28,6 +27,7 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
|
||||
import java.util.Collections;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
@ -107,7 +107,7 @@ public class CandidateStateTest {
|
|||
);
|
||||
assertFalse(state.isVoteGranted());
|
||||
assertFalse(state.isVoteRejected());
|
||||
assertEquals(Utils.mkSet(node1, node2), state.unrecordedVoters());
|
||||
assertEquals(Set.of(node1, node2), state.unrecordedVoters());
|
||||
assertTrue(state.recordGrantedVote(node1.id()));
|
||||
assertEquals(Collections.singleton(node2), state.unrecordedVoters());
|
||||
assertTrue(state.isVoteGranted());
|
||||
|
@ -128,7 +128,7 @@ public class CandidateStateTest {
|
|||
);
|
||||
assertFalse(state.isVoteGranted());
|
||||
assertFalse(state.isVoteRejected());
|
||||
assertEquals(Utils.mkSet(node1, node2), state.unrecordedVoters());
|
||||
assertEquals(Set.of(node1, node2), state.unrecordedVoters());
|
||||
assertTrue(state.recordRejectedVote(node1.id()));
|
||||
assertEquals(Collections.singleton(node2), state.unrecordedVoters());
|
||||
assertFalse(state.isVoteGranted());
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.kafka.raft;
|
||||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.generated.QuorumStateData;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -27,6 +26,7 @@ import org.junit.jupiter.params.provider.ValueSource;
|
|||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -69,20 +69,20 @@ final class ElectionStateTest {
|
|||
void testQuorumStateDataRoundTrip(short version) {
|
||||
ReplicaKey votedKey = ReplicaKey.of(1, Uuid.randomUuid());
|
||||
List<ElectionState> electionStates = Arrays.asList(
|
||||
ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withVotedCandidate(5, votedKey, Utils.mkSet(1, 2, 3))
|
||||
ElectionState.withUnknownLeader(5, Set.of(1, 2, 3)),
|
||||
ElectionState.withElectedLeader(5, 1, Set.of(1, 2, 3)),
|
||||
ElectionState.withVotedCandidate(5, votedKey, Set.of(1, 2, 3))
|
||||
);
|
||||
|
||||
final List<ElectionState> expected;
|
||||
if (version == 0) {
|
||||
expected = Arrays.asList(
|
||||
ElectionState.withUnknownLeader(5, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withElectedLeader(5, 1, Utils.mkSet(1, 2, 3)),
|
||||
ElectionState.withUnknownLeader(5, Set.of(1, 2, 3)),
|
||||
ElectionState.withElectedLeader(5, 1, Set.of(1, 2, 3)),
|
||||
ElectionState.withVotedCandidate(
|
||||
5,
|
||||
ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID),
|
||||
Utils.mkSet(1, 2, 3)
|
||||
Set.of(1, 2, 3)
|
||||
)
|
||||
);
|
||||
} else {
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.kafka.raft;
|
|||
|
||||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.protocol.types.TaggedFields;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.generated.QuorumStateData;
|
||||
import org.apache.kafka.server.common.KRaftVersion;
|
||||
import org.apache.kafka.test.TestUtils;
|
||||
|
@ -57,7 +56,7 @@ public class FileQuorumStateStoreTest {
|
|||
final int voter1 = 1;
|
||||
final int voter2 = 2;
|
||||
final int voter3 = 3;
|
||||
Set<Integer> voters = Utils.mkSet(voter1, voter2, voter3);
|
||||
Set<Integer> voters = Set.of(voter1, voter2, voter3);
|
||||
|
||||
stateStore.writeElectionState(
|
||||
ElectionState.withElectedLeader(epoch, voter1, voters),
|
||||
|
@ -88,7 +87,7 @@ public class FileQuorumStateStoreTest {
|
|||
final ReplicaKey voter1Key = ReplicaKey.of(voter1, Uuid.randomUuid());
|
||||
final int voter2 = 2;
|
||||
final int voter3 = 3;
|
||||
Set<Integer> voters = Utils.mkSet(voter1, voter2, voter3);
|
||||
Set<Integer> voters = Set.of(voter1, voter2, voter3);
|
||||
|
||||
stateStore.writeElectionState(
|
||||
ElectionState.withVotedCandidate(epoch, voter1Key, voters),
|
||||
|
@ -124,7 +123,7 @@ public class FileQuorumStateStoreTest {
|
|||
FileQuorumStateStore stateStore = new FileQuorumStateStore(TestUtils.tempFile());
|
||||
|
||||
final int epoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(1, 2, 3);
|
||||
Set<Integer> voters = Set.of(1, 2, 3);
|
||||
|
||||
stateStore.writeElectionState(
|
||||
ElectionState.withUnknownLeader(epoch, voters),
|
||||
|
@ -148,7 +147,7 @@ public class FileQuorumStateStoreTest {
|
|||
FileQuorumStateStore stateStore = new FileQuorumStateStore(stateFile);
|
||||
|
||||
final int epoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(1, 2, 3);
|
||||
Set<Integer> voters = Set.of(1, 2, 3);
|
||||
|
||||
stateStore.writeElectionState(ElectionState.withUnknownLeader(epoch, voters), KRAFT_VERSION_1);
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@ package org.apache.kafka.raft;
|
|||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -67,7 +66,7 @@ public class FollowerStateTest {
|
|||
|
||||
@Test
|
||||
public void testFetchTimeoutExpiration() {
|
||||
FollowerState state = newFollowerState(Utils.mkSet(1, 2, 3), Optional.empty());
|
||||
FollowerState state = newFollowerState(Set.of(1, 2, 3), Optional.empty());
|
||||
|
||||
assertFalse(state.hasFetchTimeoutExpired(time.milliseconds()));
|
||||
assertEquals(fetchTimeoutMs, state.remainingFetchTimeMs(time.milliseconds()));
|
||||
|
@ -83,7 +82,7 @@ public class FollowerStateTest {
|
|||
|
||||
@Test
|
||||
public void testMonotonicHighWatermark() {
|
||||
FollowerState state = newFollowerState(Utils.mkSet(1, 2, 3), Optional.empty());
|
||||
FollowerState state = newFollowerState(Set.of(1, 2, 3), Optional.empty());
|
||||
|
||||
OptionalLong highWatermark = OptionalLong.of(15L);
|
||||
state.updateHighWatermark(highWatermark);
|
||||
|
@ -97,7 +96,7 @@ public class FollowerStateTest {
|
|||
@ValueSource(booleans = {true, false})
|
||||
public void testGrantVote(boolean isLogUpToDate) {
|
||||
FollowerState state = newFollowerState(
|
||||
Utils.mkSet(1, 2, 3),
|
||||
Set.of(1, 2, 3),
|
||||
Optional.empty()
|
||||
);
|
||||
|
||||
|
@ -108,7 +107,7 @@ public class FollowerStateTest {
|
|||
|
||||
@Test
|
||||
public void testLeaderIdAndEndpoint() {
|
||||
FollowerState state = newFollowerState(Utils.mkSet(0, 1, 2), Optional.empty());
|
||||
FollowerState state = newFollowerState(Set.of(0, 1, 2), Optional.empty());
|
||||
|
||||
assertEquals(leaderId, state.leaderId());
|
||||
assertEquals(leaderEndpoints, state.leaderEndpoints());
|
||||
|
|
|
@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.Errors;
|
|||
import org.apache.kafka.common.record.MemoryRecords;
|
||||
import org.apache.kafka.common.record.UnalignedMemoryRecords;
|
||||
import org.apache.kafka.common.requests.FetchSnapshotRequest;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.internals.StringSerde;
|
||||
import org.apache.kafka.snapshot.RawSnapshotReader;
|
||||
import org.apache.kafka.snapshot.RawSnapshotWriter;
|
||||
|
@ -64,7 +63,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testLatestSnapshotId() throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
|
@ -82,7 +81,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testLatestSnapshotIdMissing() throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
|
@ -100,7 +99,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testLeaderListenerNotified(boolean entireLog, boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, false);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
RaftClientTestContext.Builder contextBuilder = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -137,7 +136,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFollowerListenerNotified(boolean entireLog) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
|
@ -179,7 +178,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testSecondListenerNotified(boolean entireLog) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
|
@ -225,7 +224,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testListenerRenotified(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 1);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -281,7 +280,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testLeaderImmediatelySendsSnapshotId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(3, 4);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -315,7 +314,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchRequestOffsetLessThanLogStart(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(1)
|
||||
|
@ -365,7 +364,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
// When the follower sends a FETCH request at offset 0, reply with snapshot id if it exists
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(1)
|
||||
|
@ -413,7 +412,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchRequestWithLargerLastFetchedEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -455,7 +454,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -505,7 +504,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -551,7 +550,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -605,7 +604,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int syncNodeId = otherNodeKey.id() + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), syncNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), syncNodeId);
|
||||
|
||||
OffsetAndEpoch oldestSnapshotId = new OffsetAndEpoch(3, 2);
|
||||
|
||||
|
@ -654,7 +653,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestMissingSnapshot(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(3)
|
||||
|
@ -717,7 +716,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestUnknownPartition(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
TopicPartition topicPartition = new TopicPartition("unknown", 0);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -748,7 +747,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
List<String> records = Arrays.asList("foo", "bar");
|
||||
|
||||
|
@ -804,7 +803,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
ReplicaKey voter1 = replicaKey(localId + 1, withKip853Rpc);
|
||||
ReplicaKey voter2 = replicaKey(localId + 2, withKip853Rpc);
|
||||
ReplicaKey observer3 = replicaKey(localId + 3, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter1.id(), voter2.id());
|
||||
Set<Integer> voters = Set.of(localId, voter1.id(), voter2.id());
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
List<String> records = Arrays.asList("foo", "bar");
|
||||
|
||||
|
@ -894,7 +893,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testPartialFetchSnapshotRequestAsLeader(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(2, 1);
|
||||
List<String> records = Arrays.asList("foo", "bar");
|
||||
|
||||
|
@ -976,7 +975,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotRequestAsFollower(boolean withKip853Rpc) throws IOException {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID;
|
||||
|
||||
|
@ -1007,7 +1006,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestWithInvalidPosition(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(1, 1);
|
||||
List<String> records = Arrays.asList("foo", "bar");
|
||||
|
||||
|
@ -1067,7 +1066,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestWithOlderEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -1100,7 +1099,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
@ValueSource(booleans = { false, true })
|
||||
public void testFetchSnapshotRequestWithNewerEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1);
|
||||
OffsetAndEpoch snapshotId = Snapshots.BOOTSTRAP_SNAPSHOT_ID;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -1134,7 +1133,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchResponseWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch invalidEpoch = new OffsetAndEpoch(100L, -1);
|
||||
OffsetAndEpoch invalidEndOffset = new OffsetAndEpoch(-1L, 1);
|
||||
|
@ -1197,7 +1196,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchResponseWithSnapshotId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1270,7 +1269,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponsePartialData(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1375,7 +1374,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponseMissingSnapshot(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1437,7 +1436,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int localId = randomReplicaId();
|
||||
int firstLeaderId = localId + 1;
|
||||
int secondLeaderId = firstLeaderId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, firstLeaderId, secondLeaderId);
|
||||
Set<Integer> voters = Set.of(localId, firstLeaderId, secondLeaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1498,7 +1497,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponseFromNewerEpochLeader(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1559,7 +1558,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponseFromOlderEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1630,7 +1629,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponseWithInvalidId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1746,7 +1745,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testFetchSnapshotResponseToNotFollower(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId);
|
||||
int epoch = 2;
|
||||
OffsetAndEpoch snapshotId = new OffsetAndEpoch(100L, 1);
|
||||
|
||||
|
@ -1822,7 +1821,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNode = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNode.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNode.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -1898,7 +1897,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
public void testCreateSnapshotAsLeaderWithInvalidSnapshotId(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
int epoch = 2;
|
||||
|
||||
List<String> appendRecords = Arrays.asList("a", "b", "c");
|
||||
|
@ -1949,7 +1948,7 @@ public final class KafkaRaftClientSnapshotTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherFollowerId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, leaderId, otherFollowerId);
|
||||
Set<Integer> voters = Set.of(localId, leaderId, otherFollowerId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, leaderId)
|
||||
|
|
|
@ -126,7 +126,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey.id());
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -158,7 +158,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey.id());
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -190,7 +190,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey.id());
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -227,7 +227,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey.id());
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -264,7 +264,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey remoteKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -305,7 +305,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInitializeAsResignedAndBecomeCandidate(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId);
|
||||
Set<Integer> voters = Set.of(localId, remoteId);
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -333,7 +333,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInitializeAsResignedLeaderFromStateStore(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId);
|
||||
Set<Integer> voters = Set.of(localId, remoteId);
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -369,7 +369,7 @@ public class KafkaRaftClientTest {
|
|||
public void testAppendFailedWithNotLeaderException(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId);
|
||||
Set<Integer> voters = Set.of(localId, remoteId);
|
||||
int epoch = 2;
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
|
@ -385,7 +385,7 @@ public class KafkaRaftClientTest {
|
|||
public void testAppendFailedWithBufferAllocationException(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
MemoryPool memoryPool = Mockito.mock(MemoryPool.class);
|
||||
ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES);
|
||||
|
@ -412,7 +412,7 @@ public class KafkaRaftClientTest {
|
|||
public void testAppendFailedWithFencedEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -433,7 +433,7 @@ public class KafkaRaftClientTest {
|
|||
public void testAppendFailedWithRecordBatchTooLargeException(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -460,7 +460,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int voter1 = localId + 1;
|
||||
int voter2 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter1, voter2);
|
||||
Set<Integer> voters = Set.of(localId, voter1, voter2);
|
||||
int epoch = 19;
|
||||
|
||||
// Start off as leader so that we will initialize in the Resigned state.
|
||||
|
@ -476,7 +476,7 @@ public class KafkaRaftClientTest {
|
|||
|
||||
context.pollUntilRequest();
|
||||
List<RaftRequest.Outbound> requests = context.collectEndQuorumRequests(
|
||||
epoch, Utils.mkSet(voter1, voter2), Optional.empty());
|
||||
epoch, Set.of(voter1, voter2), Optional.empty());
|
||||
assertEquals(2, requests.size());
|
||||
|
||||
// Respond to one of the requests so that we can verify that no additional
|
||||
|
@ -496,7 +496,7 @@ public class KafkaRaftClientTest {
|
|||
context.time.sleep(6000);
|
||||
context.pollUntilRequest();
|
||||
List<RaftRequest.Outbound> retries = context.collectEndQuorumRequests(
|
||||
epoch, Utils.mkSet(nonRespondedId), Optional.empty());
|
||||
epoch, Set.of(nonRespondedId), Optional.empty());
|
||||
assertEquals(1, retries.size());
|
||||
}
|
||||
|
||||
|
@ -506,7 +506,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId = localId + 1;
|
||||
ReplicaKey otherNodeKey = replicaKey(remoteId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -544,7 +544,7 @@ public class KafkaRaftClientTest {
|
|||
public void testResignInOlderEpochIgnored(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -571,7 +571,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId1 = localId + 1;
|
||||
int remoteId2 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId1, remoteId2);
|
||||
Set<Integer> voters = Set.of(localId, remoteId1, remoteId2);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -599,7 +599,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int remoteId1 = localId + 1;
|
||||
int remoteId2 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId1, remoteId2);
|
||||
Set<Integer> voters = Set.of(localId, remoteId1, remoteId2);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -612,16 +612,16 @@ public class KafkaRaftClientTest {
|
|||
// begin epoch requests should be sent out every beginQuorumEpochTimeoutMs
|
||||
context.time.sleep(context.beginQuorumEpochTimeoutMs);
|
||||
context.client.poll();
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Utils.mkSet(remoteId1, remoteId2));
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));
|
||||
|
||||
int partialDelay = context.beginQuorumEpochTimeoutMs / 2;
|
||||
context.time.sleep(partialDelay);
|
||||
context.client.poll();
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Utils.mkSet());
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Set.of());
|
||||
|
||||
context.time.sleep(context.beginQuorumEpochTimeoutMs - partialDelay);
|
||||
context.client.poll();
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Utils.mkSet(remoteId1, remoteId2));
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(remoteId1, remoteId2));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
|
@ -634,7 +634,7 @@ public class KafkaRaftClientTest {
|
|||
ReplicaKey remoteKey1 = replicaKey(remoteId1, withKip853Rpc);
|
||||
ReplicaKey remoteKey2 = replicaKey(remoteId2, withKip853Rpc);
|
||||
ReplicaKey observerKey3 = replicaKey(observerId, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteKey1.id(), remoteKey2.id());
|
||||
Set<Integer> voters = Set.of(localId, remoteKey1.id(), remoteKey2.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -688,7 +688,7 @@ public class KafkaRaftClientTest {
|
|||
@ValueSource(booleans = { true, false })
|
||||
public void testLeaderShouldNotResignLeadershipIfOnlyOneVoters(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(localId);
|
||||
Set<Integer> voters = Set.of(localId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -708,7 +708,7 @@ public class KafkaRaftClientTest {
|
|||
public void testElectionTimeoutAfterUserInitiatedResign(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -761,7 +761,7 @@ public class KafkaRaftClientTest {
|
|||
public void testCannotResignWithLargerEpochThanCurrentEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -778,7 +778,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int leaderEpoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(leaderEpoch, otherNodeId)
|
||||
|
@ -795,7 +795,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = randomReplicaId();
|
||||
int otherNodeId = randomReplicaId() + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(OptionalInt.empty(), voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -822,7 +822,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInitializeAsCandidateFromStateStore(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
// Need 3 node to require a 2-node majority
|
||||
Set<Integer> voters = Utils.mkSet(localId, localId + 1, localId + 2);
|
||||
Set<Integer> voters = Set.of(localId, localId + 1, localId + 2);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withVotedCandidate(2, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))
|
||||
|
@ -842,7 +842,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInitializeAsCandidateAndBecomeLeader(boolean withKip853Rpc) throws Exception {
|
||||
final int localId = randomReplicaId();
|
||||
final int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
@ -871,7 +871,7 @@ public class KafkaRaftClientTest {
|
|||
|
||||
// Send BeginQuorumEpoch to voters
|
||||
context.client.poll();
|
||||
context.assertSentBeginQuorumEpochRequest(1, Utils.mkSet(otherNodeId));
|
||||
context.assertSentBeginQuorumEpochRequest(1, Set.of(otherNodeId));
|
||||
|
||||
Records records = context.log.read(0, Isolation.UNCOMMITTED).records;
|
||||
RecordBatch batch = records.batches().iterator().next();
|
||||
|
@ -889,7 +889,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
final int firstNodeId = localId + 1;
|
||||
final int secondNodeId = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, firstNodeId, secondNodeId);
|
||||
Set<Integer> voters = Set.of(localId, firstNodeId, secondNodeId);
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
.build();
|
||||
|
@ -922,7 +922,7 @@ public class KafkaRaftClientTest {
|
|||
|
||||
// Send BeginQuorumEpoch to voters
|
||||
context.client.poll();
|
||||
context.assertSentBeginQuorumEpochRequest(1, Utils.mkSet(firstNodeId, secondNodeId));
|
||||
context.assertSentBeginQuorumEpochRequest(1, Set.of(firstNodeId, secondNodeId));
|
||||
|
||||
Records records = context.log.read(0, Isolation.UNCOMMITTED).records;
|
||||
RecordBatch batch = records.batches().iterator().next();
|
||||
|
@ -940,7 +940,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int votedCandidateEpoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withVotedCandidate(votedCandidateEpoch, otherNodeKey)
|
||||
|
@ -1004,7 +1004,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int leaderEpoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(leaderEpoch, localId)
|
||||
|
@ -1024,7 +1024,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
int jitterMs = 85;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.updateRandom(r -> r.mockNextInt(jitterMs))
|
||||
|
@ -1070,7 +1070,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int voter2 = localId + 1;
|
||||
ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2, voter3.id());
|
||||
Set<Integer> voters = Set.of(localId, voter2, voter3.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(6)
|
||||
|
@ -1102,7 +1102,7 @@ public class KafkaRaftClientTest {
|
|||
int voter2 = localId + 1;
|
||||
ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc);
|
||||
int epoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2, voter3.id());
|
||||
Set<Integer> voters = Set.of(localId, voter2, voter3.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -1131,7 +1131,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int lingerMs = 50;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
MemoryPool memoryPool = Mockito.mock(MemoryPool.class);
|
||||
ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES);
|
||||
|
@ -1164,7 +1164,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int lingerMs = 50;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
MemoryPool memoryPool = Mockito.mock(MemoryPool.class);
|
||||
ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES);
|
||||
|
@ -1198,7 +1198,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int lingerMs = 50;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
MemoryPool memoryPool = Mockito.mock(MemoryPool.class);
|
||||
ByteBuffer buffer = ByteBuffer.allocate(KafkaRaftClient.MAX_BATCH_SIZE_BYTES);
|
||||
|
@ -1233,7 +1233,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int lingerMs = 50;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(lingerMs)
|
||||
|
@ -1269,7 +1269,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int lingerMs = 50;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withAppendLingerMs(lingerMs)
|
||||
|
@ -1304,7 +1304,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int oldLeaderId = localId + 1;
|
||||
int leaderEpoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, oldLeaderId);
|
||||
Set<Integer> voters = Set.of(localId, oldLeaderId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(leaderEpoch, oldLeaderId)
|
||||
|
@ -1333,7 +1333,7 @@ public class KafkaRaftClientTest {
|
|||
ReplicaKey oldLeaderKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int leaderEpoch = 2;
|
||||
ReplicaKey preferredNextLeader = replicaKey(localId + 2, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, oldLeaderKey.id(), preferredNextLeader.id());
|
||||
Set<Integer> voters = Set.of(localId, oldLeaderKey.id(), preferredNextLeader.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(leaderEpoch, oldLeaderKey.id())
|
||||
|
@ -1375,7 +1375,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -1417,7 +1417,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int epoch = 2;
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -1439,7 +1439,7 @@ public class KafkaRaftClientTest {
|
|||
int epoch = 2;
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int electedLeaderId = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), electedLeaderId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), electedLeaderId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, electedLeaderId)
|
||||
|
@ -1461,7 +1461,7 @@ public class KafkaRaftClientTest {
|
|||
int epoch = 2;
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
ReplicaKey votedCandidateKey = replicaKey(localId + 2, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id(), votedCandidateKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id(), votedCandidateKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withVotedCandidate(epoch, votedCandidateKey)
|
||||
|
@ -1481,7 +1481,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int epoch = 2;
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -1502,7 +1502,7 @@ public class KafkaRaftClientTest {
|
|||
int epoch = 2;
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int otherNodeId2 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(otherNodeKey.id(), otherNodeId2);
|
||||
Set<Integer> voters = Set.of(otherNodeKey.id(), otherNodeId2);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -1521,7 +1521,7 @@ public class KafkaRaftClientTest {
|
|||
public void testLeaderIgnoreVoteRequestOnSameEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(2)
|
||||
|
@ -1544,7 +1544,7 @@ public class KafkaRaftClientTest {
|
|||
public void testListenerCommitCallbackAfterLeaderWrite(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -1590,7 +1590,7 @@ public class KafkaRaftClientTest {
|
|||
public void testLeaderImmediatelySendsDivergingEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(5)
|
||||
|
@ -1623,7 +1623,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int leaderEpoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withVotedCandidate(leaderEpoch, ReplicaKey.of(localId, ReplicaKey.NO_DIRECTORY_ID))
|
||||
|
@ -1645,7 +1645,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 1;
|
||||
int exponentialFactor = 85; // set it large enough so that we will bound on jitter
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.updateRandom(r -> r.mockNextInt(exponentialFactor))
|
||||
|
@ -1691,7 +1691,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -1712,7 +1712,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
int lastEpoch = 3;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -1733,7 +1733,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
int lastEpoch = 3;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -1758,7 +1758,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
int lastEpoch = 3;
|
||||
Set<Integer> voters = Utils.mkSet(otherNodeId);
|
||||
Set<Integer> voters = Set.of(otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -1787,7 +1787,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(otherNodeId);
|
||||
Set<Integer> voters = Set.of(otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -1825,7 +1825,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -1852,7 +1852,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int leaderId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId);
|
||||
Set<Integer> voters = Set.of(leaderId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -1899,7 +1899,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -1942,7 +1942,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -2016,7 +2016,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -2071,7 +2071,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInvalidFetchRequest(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2118,7 +2118,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
ReplicaKey otherNodeKey = replicaKey(otherNodeId, false);
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = RaftClientTestContext.initializeAsLeader(localId, voters, epoch);
|
||||
|
||||
|
@ -2143,7 +2143,7 @@ public class KafkaRaftClientTest {
|
|||
public void testFetchRequestClusterIdValidation(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2179,7 +2179,7 @@ public class KafkaRaftClientTest {
|
|||
public void testVoteRequestClusterIdValidation(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -2213,7 +2213,7 @@ public class KafkaRaftClientTest {
|
|||
public void testInvalidVoterReplicaVoteRequest() throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, true);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(true)
|
||||
|
@ -2257,7 +2257,7 @@ public class KafkaRaftClientTest {
|
|||
int voter2 = localId + 1;
|
||||
int voter3 = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2, voter3);
|
||||
Set<Integer> voters = Set.of(localId, voter2, voter3);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch - 1)
|
||||
|
@ -2309,7 +2309,7 @@ public class KafkaRaftClientTest {
|
|||
public void testBeginQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2345,7 +2345,7 @@ public class KafkaRaftClientTest {
|
|||
public void testEndQuorumEpochRequestClusterIdValidation(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2381,7 +2381,7 @@ public class KafkaRaftClientTest {
|
|||
public void testLeaderAcceptVoteFromObserver(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2407,7 +2407,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeKey.id())
|
||||
|
@ -2451,7 +2451,7 @@ public class KafkaRaftClientTest {
|
|||
public void testPurgatoryFetchTimeout(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2479,7 +2479,7 @@ public class KafkaRaftClientTest {
|
|||
public void testPurgatoryFetchSatisfiedByWrite(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2510,7 +2510,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey voterKey2 = replicaKey(localId + 1, withKip853Rpc);
|
||||
int voter3 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voterKey2.id(), voter3);
|
||||
Set<Integer> voters = Set.of(localId, voterKey2.id(), voter3);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(4)
|
||||
|
@ -2547,7 +2547,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
// The other node starts out as the leader
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -2587,7 +2587,7 @@ public class KafkaRaftClientTest {
|
|||
int voter3 = localId + 2;
|
||||
int epoch = 5;
|
||||
// Start out with `voter2` as the leader
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2, voter3);
|
||||
Set<Integer> voters = Set.of(localId, voter2, voter3);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, voter2)
|
||||
|
@ -2625,7 +2625,7 @@ public class KafkaRaftClientTest {
|
|||
int voter2 = localId + 1;
|
||||
int voter3 = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2, voter3);
|
||||
Set<Integer> voters = Set.of(localId, voter2, voter3);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch - 1)
|
||||
|
@ -2673,7 +2673,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -2724,7 +2724,7 @@ public class KafkaRaftClientTest {
|
|||
int leaderId = localId + 1;
|
||||
int otherNodeId = localId + 2;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(leaderId, otherNodeId);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
@ -2767,7 +2767,7 @@ public class KafkaRaftClientTest {
|
|||
public void testLeaderGracefulShutdown(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -2815,7 +2815,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey closeFollower = replicaKey(localId + 2, withKip853Rpc);
|
||||
ReplicaKey laggingFollower = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, closeFollower.id(), laggingFollower.id());
|
||||
Set<Integer> voters = Set.of(localId, closeFollower.id(), laggingFollower.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -2850,7 +2850,7 @@ public class KafkaRaftClientTest {
|
|||
|
||||
context.collectEndQuorumRequests(
|
||||
epoch,
|
||||
Utils.mkSet(closeFollower.id(), laggingFollower.id()),
|
||||
Set.of(closeFollower.id(), laggingFollower.id()),
|
||||
Optional.of(
|
||||
Arrays.asList(
|
||||
replicaKey(closeFollower.id(), false),
|
||||
|
@ -2867,7 +2867,7 @@ public class KafkaRaftClientTest {
|
|||
ReplicaKey voter2 = replicaKey(localId + 1, withKip853Rpc);
|
||||
ReplicaKey voter3 = replicaKey(localId + 2, withKip853Rpc);
|
||||
int epoch = 2;
|
||||
Set<Integer> voters = Utils.mkSet(localId, voter2.id(), voter3.id());
|
||||
Set<Integer> voters = Set.of(localId, voter2.id(), voter3.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -2897,7 +2897,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey local = replicaKey(localId, true);
|
||||
ReplicaKey follower1 = replicaKey(localId + 1, true);
|
||||
Set<Integer> voters = Utils.mkSet(localId, follower1.id());
|
||||
Set<Integer> voters = Set.of(localId, follower1.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get())
|
||||
.withStaticVoters(voters)
|
||||
|
@ -3265,7 +3265,7 @@ public class KafkaRaftClientTest {
|
|||
ReplicaKey follower = replicaKey(localId + 1, true);
|
||||
ReplicaKey follower2 = replicaKey(localId + 2, true);
|
||||
// only include one follower in static voter set
|
||||
Set<Integer> staticVoters = Utils.mkSet(localId, follower.id());
|
||||
Set<Integer> staticVoters = Set.of(localId, follower.id());
|
||||
VoterSet voterSet = VoterSetTest.voterSet(Stream.of(local, follower, follower2));
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, local.directoryId().get())
|
||||
|
@ -3307,7 +3307,7 @@ public class KafkaRaftClientTest {
|
|||
public void testLeaderGracefulShutdownTimeout(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(1)
|
||||
|
@ -3346,7 +3346,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -3373,7 +3373,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int voter1 = localId + 1;
|
||||
int voter2 = localId + 2;
|
||||
Set<Integer> voters = Utils.mkSet(voter1, voter2);
|
||||
Set<Integer> voters = Set.of(voter1, voter2);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(5)
|
||||
|
@ -3418,7 +3418,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -3448,7 +3448,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(otherNodeId);
|
||||
Set<Integer> voters = Set.of(otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -3480,7 +3480,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -3547,7 +3547,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0))
|
||||
|
@ -3561,7 +3561,7 @@ public class KafkaRaftClientTest {
|
|||
context.pollUntilRequest();
|
||||
|
||||
// We send BeginEpoch, but it gets lost and the destination finds the leader through the Fetch API
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Utils.mkSet(otherNodeKey.id()));
|
||||
context.assertSentBeginQuorumEpochRequest(epoch, Set.of(otherNodeKey.id()));
|
||||
|
||||
context.deliverRequest(context.fetchRequest(epoch, otherNodeKey, 0L, 0, 500));
|
||||
|
||||
|
@ -3661,7 +3661,7 @@ public class KafkaRaftClientTest {
|
|||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
int lastEpoch = 3;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -3745,7 +3745,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -3773,7 +3773,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.updateRandom(r -> r.mockNextInt(DEFAULT_ELECTION_TIMEOUT_MS, 0))
|
||||
|
@ -3802,7 +3802,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch - 1)
|
||||
|
@ -3827,7 +3827,7 @@ public class KafkaRaftClientTest {
|
|||
public void testClusterAuthorizationFailedInEndQuorumEpoch(boolean withKip853Rpc) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(1)
|
||||
|
@ -3855,7 +3855,7 @@ public class KafkaRaftClientTest {
|
|||
) throws Exception {
|
||||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withKip853Rpc(withKip853Rpc)
|
||||
|
@ -3903,7 +3903,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
List<String> batch1 = Arrays.asList("1", "2", "3");
|
||||
List<String> batch2 = Arrays.asList("4", "5", "6");
|
||||
|
@ -3970,7 +3970,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
List<String> batch1 = Arrays.asList("1", "2", "3");
|
||||
List<String> batch2 = Arrays.asList("4", "5", "6");
|
||||
|
@ -4013,7 +4013,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
List<String> batch1 = Arrays.asList("1", "2", "3");
|
||||
List<String> batch2 = Arrays.asList("4", "5", "6");
|
||||
|
@ -4057,7 +4057,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 5;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -4118,7 +4118,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 7;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(2, Arrays.asList("a", "b", "c"))
|
||||
|
@ -4165,7 +4165,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
ReplicaKey otherNodeKey = replicaKey(localId + 1, withKip853Rpc);
|
||||
int epoch = 7;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeKey.id());
|
||||
Set<Integer> voters = Set.of(localId, otherNodeKey.id());
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.appendToLog(2, Arrays.asList("a", "b", "c"))
|
||||
|
@ -4223,7 +4223,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 7;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withUnknownLeader(epoch)
|
||||
|
@ -4258,7 +4258,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 7;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, otherNodeId)
|
||||
|
@ -4284,7 +4284,7 @@ public class KafkaRaftClientTest {
|
|||
int localId = randomReplicaId();
|
||||
int otherNodeId = localId + 1;
|
||||
int epoch = 7;
|
||||
Set<Integer> voters = Utils.mkSet(localId, otherNodeId);
|
||||
Set<Integer> voters = Set.of(localId, otherNodeId);
|
||||
|
||||
RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters)
|
||||
.withElectedLeader(epoch, localId)
|
||||
|
@ -4312,7 +4312,7 @@ public class KafkaRaftClientTest {
|
|||
// When no `localId` is defined, the client will behave as an observer.
|
||||
// This is designed for tooling/debugging use cases.
|
||||
int leaderId = randomReplicaId();
|
||||
Set<Integer> voters = Utils.mkSet(leaderId, leaderId + 1);
|
||||
Set<Integer> voters = Set.of(leaderId, leaderId + 1);
|
||||
List<InetSocketAddress> bootstrapServers = voters
|
||||
.stream()
|
||||
.map(RaftClientTestContext::mockAddress)
|
||||
|
|
|
@ -33,12 +33,12 @@ import java.util.Map;
|
|||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.raft.LeaderState.CHECK_QUORUM_TIMEOUT_FACTOR;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
@ -132,7 +132,7 @@ public class LeaderStateTest {
|
|||
localWithRemoteVoterSet(Stream.of(node1, node2), withDirectoryId),
|
||||
0L
|
||||
);
|
||||
assertEquals(mkSet(node1, node2), state.nonAcknowledgingVoters());
|
||||
assertEquals(Set.of(node1, node2), state.nonAcknowledgingVoters());
|
||||
state.addAcknowledgementFrom(node1.id());
|
||||
assertEquals(singleton(node2), state.nonAcknowledgingVoters());
|
||||
state.addAcknowledgementFrom(node2.id());
|
||||
|
@ -233,7 +233,7 @@ public class LeaderStateTest {
|
|||
LeaderState<?> state = newLeaderState(voters, 10L);
|
||||
|
||||
assertFalse(state.updateLocalState(new LogOffsetMetadata(15L), voters));
|
||||
assertEquals(mkSet(nodeKey1, nodeKey2), state.nonAcknowledgingVoters());
|
||||
assertEquals(Set.of(nodeKey1, nodeKey2), state.nonAcknowledgingVoters());
|
||||
assertEquals(Optional.empty(), state.highWatermark());
|
||||
assertFalse(state.updateReplicaState(nodeKey1, 0, new LogOffsetMetadata(10L)));
|
||||
assertEquals(singleton(nodeKey2), state.nonAcknowledgingVoters());
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.Uuid;
|
|||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.internals.BatchAccumulator;
|
||||
import org.apache.kafka.raft.internals.KRaftControlRecordStateMachine;
|
||||
import org.apache.kafka.server.common.Features;
|
||||
|
@ -293,8 +292,8 @@ public class QuorumStateTest {
|
|||
ElectionState.withVotedCandidate(epoch, localVoterKey, voters.voterIds()),
|
||||
candidateState.election()
|
||||
);
|
||||
assertEquals(Utils.mkSet(node1, node2), candidateState.unrecordedVoters());
|
||||
assertEquals(Utils.mkSet(localId), candidateState.grantingVoters());
|
||||
assertEquals(Set.of(node1, node2), candidateState.unrecordedVoters());
|
||||
assertEquals(Set.of(localId), candidateState.grantingVoters());
|
||||
assertEquals(Collections.emptySet(), candidateState.rejectingVoters());
|
||||
assertEquals(
|
||||
electionTimeoutMs + jitterMs,
|
||||
|
@ -328,7 +327,7 @@ public class QuorumStateTest {
|
|||
ResignedState resignedState = state.resignedStateOrThrow();
|
||||
assertEquals(epoch, resignedState.epoch());
|
||||
assertEquals(election, resignedState.election());
|
||||
assertEquals(Utils.mkSet(node1, node2), resignedState.unackedVoters());
|
||||
assertEquals(Set.of(node1, node2), resignedState.unackedVoters());
|
||||
assertEquals(electionTimeoutMs + jitterMs,
|
||||
resignedState.remainingElectionTimeMs(time.milliseconds()));
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import org.apache.kafka.common.utils.BufferSupplier;
|
|||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.raft.MockLog.LogBatch;
|
||||
import org.apache.kafka.raft.MockLog.LogEntry;
|
||||
import org.apache.kafka.raft.internals.BatchMemoryPool;
|
||||
|
@ -260,21 +259,21 @@ public class RaftEventSimulationTest {
|
|||
// to make progress even if an election is needed in the larger set.
|
||||
router.filter(
|
||||
0,
|
||||
new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(2, 3, 4)))
|
||||
new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(2, 3, 4)))
|
||||
);
|
||||
router.filter(
|
||||
1,
|
||||
new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(2, 3, 4)))
|
||||
new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(2, 3, 4)))
|
||||
);
|
||||
router.filter(2, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1))));
|
||||
router.filter(3, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1))));
|
||||
router.filter(4, new DropOutboundRequestsTo(cluster.endpointsFromIds(Utils.mkSet(0, 1))));
|
||||
router.filter(2, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0, 1))));
|
||||
router.filter(3, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0, 1))));
|
||||
router.filter(4, new DropOutboundRequestsTo(cluster.endpointsFromIds(Set.of(0, 1))));
|
||||
|
||||
long partitionLogEndOffset = cluster.maxLogEndOffset();
|
||||
scheduler.runUntil(() -> cluster.anyReachedHighWatermark(2 * partitionLogEndOffset));
|
||||
|
||||
long minorityHighWatermark = cluster.maxHighWatermarkReached(Utils.mkSet(0, 1));
|
||||
long majorityHighWatermark = cluster.maxHighWatermarkReached(Utils.mkSet(2, 3, 4));
|
||||
long minorityHighWatermark = cluster.maxHighWatermarkReached(Set.of(0, 1));
|
||||
long majorityHighWatermark = cluster.maxHighWatermarkReached(Set.of(2, 3, 4));
|
||||
|
||||
assertTrue(
|
||||
majorityHighWatermark > minorityHighWatermark,
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.kafka.raft;
|
|||
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -64,7 +63,7 @@ class ResignedStateTest {
|
|||
@Test
|
||||
public void testResignedState() {
|
||||
int remoteId = 1;
|
||||
Set<Integer> voters = Utils.mkSet(localId, remoteId);
|
||||
Set<Integer> voters = Set.of(localId, remoteId);
|
||||
|
||||
ResignedState state = newResignedState(voters);
|
||||
|
||||
|
@ -88,7 +87,7 @@ class ResignedStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testGrantVote(boolean isLogUpToDate) {
|
||||
ResignedState state = newResignedState(Utils.mkSet(1, 2, 3));
|
||||
ResignedState state = newResignedState(Set.of(1, 2, 3));
|
||||
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
assertFalse(state.canGrantVote(ReplicaKey.of(2, ReplicaKey.NO_DIRECTORY_ID), isLogUpToDate));
|
||||
|
@ -97,7 +96,7 @@ class ResignedStateTest {
|
|||
|
||||
@Test
|
||||
void testNegativeScenarioAcknowledgeResignation() {
|
||||
Set<Integer> voters = Utils.mkSet(0, 1, 2, 3, 4, 5);
|
||||
Set<Integer> voters = Set.of(0, 1, 2, 3, 4, 5);
|
||||
|
||||
ResignedState state = newResignedState(voters);
|
||||
|
||||
|
@ -110,7 +109,7 @@ class ResignedStateTest {
|
|||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
ResignedState state = newResignedState(Utils.mkSet(1, 2, 3));
|
||||
ResignedState state = newResignedState(Set.of(1, 2, 3));
|
||||
|
||||
assertEquals(localEndpoints, state.leaderEndpoints());
|
||||
assertNotEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.kafka.raft;
|
|||
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
|
@ -57,7 +56,7 @@ public class UnattachedStateTest {
|
|||
|
||||
@Test
|
||||
public void testElectionTimeout() {
|
||||
Set<Integer> voters = Utils.mkSet(1, 2, 3);
|
||||
Set<Integer> voters = Set.of(1, 2, 3);
|
||||
|
||||
UnattachedState state = newUnattachedState(voters, OptionalInt.empty());
|
||||
|
||||
|
@ -79,7 +78,7 @@ public class UnattachedStateTest {
|
|||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testGrantVote(boolean isLogUpToDate) {
|
||||
UnattachedState state = newUnattachedState(Utils.mkSet(1, 2, 3), OptionalInt.empty());
|
||||
UnattachedState state = newUnattachedState(Set.of(1, 2, 3), OptionalInt.empty());
|
||||
|
||||
assertEquals(
|
||||
isLogUpToDate,
|
||||
|
@ -97,7 +96,7 @@ public class UnattachedStateTest {
|
|||
|
||||
@Test
|
||||
void testLeaderEndpoints() {
|
||||
UnattachedState state = newUnattachedState(Utils.mkSet(1, 2, 3), OptionalInt.empty());
|
||||
UnattachedState state = newUnattachedState(Set.of(1, 2, 3), OptionalInt.empty());
|
||||
|
||||
assertEquals(Endpoints.empty(), state.leaderEndpoints());
|
||||
}
|
||||
|
@ -106,7 +105,7 @@ public class UnattachedStateTest {
|
|||
@ValueSource(booleans = {true, false})
|
||||
void testUnattachedWithLeader(boolean isLogUpToDate) {
|
||||
int leaderId = 3;
|
||||
Set<Integer> voters = Utils.mkSet(1, 2, leaderId);
|
||||
Set<Integer> voters = Set.of(1, 2, leaderId);
|
||||
|
||||
UnattachedState state = newUnattachedState(voters, OptionalInt.of(leaderId));
|
||||
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.Node;
|
|||
import org.apache.kafka.common.Uuid;
|
||||
import org.apache.kafka.common.feature.SupportedVersionRange;
|
||||
import org.apache.kafka.common.network.ListenerName;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.server.common.Features;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
@ -34,6 +33,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
@ -67,7 +67,7 @@ public final class VoterSetTest {
|
|||
VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true));
|
||||
|
||||
assertEquals(
|
||||
Utils.mkSet(new Node(1, "localhost", 9991), new Node(2, "localhost", 9992)),
|
||||
Set.of(new Node(1, "localhost", 9991), new Node(2, "localhost", 9992)),
|
||||
voterSet.voterNodes(IntStream.of(1, 2).boxed(), DEFAULT_LISTENER_NAME)
|
||||
);
|
||||
|
||||
|
|
|
@ -26,13 +26,13 @@ import org.junit.jupiter.api.Test;
|
|||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
@ -102,7 +102,7 @@ public class TimerTest {
|
|||
}
|
||||
});
|
||||
|
||||
assertEquals(mkSet(-5, -4, -3, -2, -1), new HashSet<>(output),
|
||||
assertEquals(Set.of(-5, -4, -3, -2, -1), new HashSet<>(output),
|
||||
"output of already expired tasks");
|
||||
}
|
||||
|
||||
|
|
|
@ -205,13 +205,13 @@ public class LogConfig extends AbstractConfig {
|
|||
private static final String MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC = TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC;
|
||||
|
||||
// Visible for testing
|
||||
public static final Set<String> CONFIGS_WITH_NO_SERVER_DEFAULTS = Collections.unmodifiableSet(Utils.mkSet(
|
||||
public static final Set<String> CONFIGS_WITH_NO_SERVER_DEFAULTS = Set.of(
|
||||
TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG,
|
||||
TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG,
|
||||
TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG,
|
||||
QuotaConfigs.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG,
|
||||
QuotaConfigs.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG
|
||||
));
|
||||
);
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private static final String MESSAGE_FORMAT_VERSION_DOC = TopicConfig.MESSAGE_FORMAT_VERSION_DOC;
|
||||
|
|
|
@ -20,7 +20,6 @@ import org.apache.kafka.common.serialization.LongSerializer;
|
|||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.serialization.StringSerializer;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.Topology.AutoOffsetReset;
|
||||
import org.apache.kafka.streams.errors.TopologyException;
|
||||
import org.apache.kafka.streams.kstream.Branched;
|
||||
|
@ -75,6 +74,7 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
|
@ -263,7 +263,7 @@ public class StreamsBuilderTest {
|
|||
equalTo(2));
|
||||
assertThat(
|
||||
topology.processorConnectedStateStores("KSTREAM-JOIN-0000000010"),
|
||||
equalTo(Utils.mkSet(topology.stateStores().get(0).name(), topology.stateStores().get(1).name())));
|
||||
equalTo(Set.of(topology.stateStores().get(0).name(), topology.stateStores().get(1).name())));
|
||||
assertTrue(
|
||||
topology.processorConnectedStateStores("KTABLE-MERGE-0000000007").isEmpty());
|
||||
}
|
||||
|
|
|
@ -62,7 +62,6 @@ import java.util.Set;
|
|||
import static java.util.Collections.nCopies;
|
||||
import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED;
|
||||
import static org.apache.kafka.common.IsolationLevel.READ_UNCOMMITTED;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.StreamsConfig.AT_LEAST_ONCE;
|
||||
import static org.apache.kafka.streams.StreamsConfig.DEFAULT_DSL_STORE_CONFIG;
|
||||
import static org.apache.kafka.streams.StreamsConfig.DSL_STORE_SUPPLIERS_CLASS_CONFIG;
|
||||
|
@ -978,7 +977,7 @@ public class StreamsConfigTest {
|
|||
props.put(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG, "cluster,zone");
|
||||
final StreamsConfig config = new StreamsConfig(props);
|
||||
assertEquals(new HashSet<>(config.getList(StreamsConfig.RACK_AWARE_ASSIGNMENT_TAGS_CONFIG)),
|
||||
mkSet("cluster", "zone"));
|
||||
Set.of("cluster", "zone"));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -71,7 +71,6 @@ import static org.apache.kafka.common.utils.Utils.mkEntry;
|
|||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
@ -125,14 +124,14 @@ public class HighAvailabilityTaskAssignorIntegrationTest {
|
|||
final String testId = safeUniqueTestName(testInfo).replaceAll("balance_subtopology", "balance");
|
||||
final String appId = "appId_" + System.currentTimeMillis() + "_" + testId;
|
||||
final String inputTopic = "input" + testId;
|
||||
final Set<TopicPartition> inputTopicPartitions = mkSet(
|
||||
final Set<TopicPartition> inputTopicPartitions = Set.of(
|
||||
new TopicPartition(inputTopic, 0),
|
||||
new TopicPartition(inputTopic, 1)
|
||||
);
|
||||
|
||||
final String storeName = "store" + testId;
|
||||
final String storeChangelog = appId + "-store" + testId + "-changelog";
|
||||
final Set<TopicPartition> changelogTopicPartitions = mkSet(
|
||||
final Set<TopicPartition> changelogTopicPartitions = Set.of(
|
||||
new TopicPartition(storeChangelog, 0),
|
||||
new TopicPartition(storeChangelog, 1)
|
||||
);
|
||||
|
|
|
@ -78,7 +78,6 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -211,7 +210,7 @@ public class IQv2IntegrationTest {
|
|||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME).withQuery(query).requireActive();
|
||||
final Set<Integer> partitions = mkSet(0, 1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
|
||||
kafkaStreams.start();
|
||||
|
||||
|
@ -279,7 +278,7 @@ public class IQv2IntegrationTest {
|
|||
@Test
|
||||
public void shouldFetchExplicitlyFromAllPartitions() {
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final Set<Integer> partitions = mkSet(0, 1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME).withQuery(query).withAllPartitions();
|
||||
|
||||
|
|
|
@ -106,7 +106,6 @@ import java.util.stream.Stream;
|
|||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
|
@ -1013,7 +1012,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet(1)
|
||||
Set.of(1)
|
||||
);
|
||||
|
||||
// miss the window start range
|
||||
|
@ -1022,7 +1021,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the window key query at the first window and the key of record which we want to query is 2
|
||||
|
@ -1031,7 +1030,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// miss the key
|
||||
|
@ -1040,7 +1039,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// miss both
|
||||
|
@ -1049,7 +1048,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the window key query at the first and the second windows and the key of record which we want to query is 0
|
||||
|
@ -1058,7 +1057,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
extractor,
|
||||
mkSet(1)
|
||||
Set.of(1)
|
||||
);
|
||||
|
||||
// do the window key query at the first window and the key of record which we want to query is 1
|
||||
|
@ -1067,7 +1066,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet(2)
|
||||
Set.of(2)
|
||||
);
|
||||
|
||||
// do the window key query at the second and the third windows and the key of record which we want to query is 2
|
||||
|
@ -1076,7 +1075,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(4, 5)
|
||||
Set.of(4, 5)
|
||||
);
|
||||
|
||||
// do the window key query at the second and the third windows and the key of record which we want to query is 3
|
||||
|
@ -1085,7 +1084,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(13)
|
||||
Set.of(13)
|
||||
);
|
||||
|
||||
// do the window key query at the fourth and the fifth windows and the key of record which we want to query is 4
|
||||
|
@ -1094,7 +1093,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(15).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(20).toMillis()),
|
||||
extractor,
|
||||
mkSet(17)
|
||||
Set.of(17)
|
||||
);
|
||||
|
||||
// do the window key query at the fifth window and the key of record which we want to query is 4
|
||||
|
@ -1103,7 +1102,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(20).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(24).toMillis()),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -1115,7 +1114,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet(1)
|
||||
Set.of(1)
|
||||
);
|
||||
|
||||
// miss the window start range
|
||||
|
@ -1124,7 +1123,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the window key query at the first window and the key of record which we want to query is 2
|
||||
|
@ -1133,7 +1132,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// miss the key
|
||||
|
@ -1142,7 +1141,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// miss both
|
||||
|
@ -1151,7 +1150,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
Instant.ofEpochMilli(WINDOW_START - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the window key query at the first and the second windows and the key of record which we want to query is 0
|
||||
|
@ -1160,7 +1159,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
extractor,
|
||||
mkSet(1)
|
||||
Set.of(1)
|
||||
);
|
||||
|
||||
// do the window key query at the first window and the key of record which we want to query is 1
|
||||
|
@ -1169,7 +1168,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START),
|
||||
Instant.ofEpochMilli(WINDOW_START),
|
||||
extractor,
|
||||
mkSet(2)
|
||||
Set.of(2)
|
||||
);
|
||||
|
||||
// do the window key query at the second and the third windows and the key of record which we want to query is 2
|
||||
|
@ -1178,7 +1177,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(4, 5)
|
||||
Set.of(4, 5)
|
||||
);
|
||||
|
||||
// do the window key query at the second and the third windows and the key of record which we want to query is 3
|
||||
|
@ -1187,7 +1186,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(7)
|
||||
Set.of(7)
|
||||
);
|
||||
|
||||
// do the window key query at the fourth and the fifth windows and the key of record which we want to query is 4
|
||||
|
@ -1196,7 +1195,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(15).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(20).toMillis()),
|
||||
extractor,
|
||||
mkSet(9)
|
||||
Set.of(9)
|
||||
);
|
||||
|
||||
// do the window key query at the fifth window and the key of record which we want to query is 4
|
||||
|
@ -1205,7 +1204,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(20).toMillis()),
|
||||
Instant.ofEpochMilli(WINDOW_START + Duration.ofMinutes(24).toMillis()),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -1218,7 +1217,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart - 1),
|
||||
Instant.ofEpochMilli(windowStart - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the query at the first window
|
||||
|
@ -1226,7 +1225,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart),
|
||||
extractor,
|
||||
mkSet(1, 2)
|
||||
Set.of(1, 2)
|
||||
);
|
||||
|
||||
// do the query at the first and the second windows
|
||||
|
@ -1234,7 +1233,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
extractor,
|
||||
mkSet(1, 2, 3, 4)
|
||||
Set.of(1, 2, 3, 4)
|
||||
);
|
||||
|
||||
// do the query at the second and the third windows
|
||||
|
@ -1242,7 +1241,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(3, 4, 5, 13)
|
||||
Set.of(3, 4, 5, 13)
|
||||
);
|
||||
|
||||
// do the query at the third and the fourth windows
|
||||
|
@ -1250,7 +1249,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(10).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(17, 5, 13)
|
||||
Set.of(17, 5, 13)
|
||||
);
|
||||
|
||||
// do the query at the fourth and the fifth windows
|
||||
|
@ -1258,7 +1257,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(20).toMillis()),
|
||||
extractor,
|
||||
mkSet(17)
|
||||
Set.of(17)
|
||||
);
|
||||
|
||||
//do the query at the fifth and the sixth windows
|
||||
|
@ -1266,7 +1265,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(20).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(25).toMillis()),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the query from the second to the fourth windows
|
||||
|
@ -1274,7 +1273,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(17, 3, 4, 5, 13)
|
||||
Set.of(17, 3, 4, 5, 13)
|
||||
);
|
||||
|
||||
// do the query from the first to the fourth windows
|
||||
|
@ -1282,7 +1281,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(1, 17, 2, 3, 4, 5, 13)
|
||||
Set.of(1, 17, 2, 3, 4, 5, 13)
|
||||
);
|
||||
|
||||
// Should fail to execute this query on a WindowStore.
|
||||
|
@ -1291,7 +1290,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, T>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, T>> result =
|
||||
|
@ -1331,7 +1330,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart - 1),
|
||||
Instant.ofEpochMilli(windowStart - 1),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the query at the first window
|
||||
|
@ -1339,7 +1338,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart),
|
||||
extractor,
|
||||
mkSet(1, 2)
|
||||
Set.of(1, 2)
|
||||
);
|
||||
|
||||
// do the query at the first and the second windows
|
||||
|
@ -1347,7 +1346,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
extractor,
|
||||
mkSet(1, 2, 3, 4)
|
||||
Set.of(1, 2, 3, 4)
|
||||
);
|
||||
|
||||
// do the query at the second and the third windows
|
||||
|
@ -1355,7 +1354,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(10).toMillis()),
|
||||
extractor,
|
||||
mkSet(3, 4, 5, 7)
|
||||
Set.of(3, 4, 5, 7)
|
||||
);
|
||||
|
||||
// do the query at the third and the fourth windows
|
||||
|
@ -1363,7 +1362,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(10).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(5, 7, 9)
|
||||
Set.of(5, 7, 9)
|
||||
);
|
||||
|
||||
// do the query at the fourth and the fifth windows
|
||||
|
@ -1371,7 +1370,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(20).toMillis()),
|
||||
extractor,
|
||||
mkSet(9)
|
||||
Set.of(9)
|
||||
);
|
||||
|
||||
//do the query at the fifth and the sixth windows
|
||||
|
@ -1379,7 +1378,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(20).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(25).toMillis()),
|
||||
extractor,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// do the query from the second to the fourth windows
|
||||
|
@ -1387,7 +1386,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(5).toMillis()),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(3, 4, 5, 7, 9)
|
||||
Set.of(3, 4, 5, 7, 9)
|
||||
);
|
||||
|
||||
// do the query from the first to the fourth windows
|
||||
|
@ -1395,7 +1394,7 @@ public class IQv2StoreIntegrationTest {
|
|||
Instant.ofEpochMilli(windowStart),
|
||||
Instant.ofEpochMilli(windowStart + Duration.ofMinutes(15).toMillis()),
|
||||
extractor,
|
||||
mkSet(1, 2, 3, 4, 5, 7, 9)
|
||||
Set.of(1, 2, 3, 4, 5, 7, 9)
|
||||
);
|
||||
|
||||
// Should fail to execute this query on a WindowStore.
|
||||
|
@ -1404,7 +1403,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, T>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, T>> result =
|
||||
|
@ -1438,33 +1437,33 @@ public class IQv2StoreIntegrationTest {
|
|||
private <T> void shouldHandleSessionKeyDSLQueries() {
|
||||
shouldHandleSessionRangeQuery(
|
||||
0,
|
||||
mkSet(1)
|
||||
Set.of(1)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
1,
|
||||
mkSet(5)
|
||||
Set.of(5)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
2,
|
||||
mkSet(9)
|
||||
Set.of(9)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
3,
|
||||
mkSet(13)
|
||||
Set.of(13)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
4,
|
||||
mkSet(17)
|
||||
Set.of(17)
|
||||
);
|
||||
|
||||
// not preset, so empty result iter
|
||||
shouldHandleSessionRangeQuery(
|
||||
999,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// Should fail to execute this query on a SessionStore.
|
||||
|
@ -1477,7 +1476,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, T>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, T>> result =
|
||||
|
@ -1511,33 +1510,33 @@ public class IQv2StoreIntegrationTest {
|
|||
private <T> void shouldHandleSessionKeyPAPIQueries() {
|
||||
shouldHandleSessionRangeQuery(
|
||||
0,
|
||||
mkSet(0, 1)
|
||||
Set.of(0, 1)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
1,
|
||||
mkSet(2, 3)
|
||||
Set.of(2, 3)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
2,
|
||||
mkSet(4, 5)
|
||||
Set.of(4, 5)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
3,
|
||||
mkSet(6, 7)
|
||||
Set.of(6, 7)
|
||||
);
|
||||
|
||||
shouldHandleSessionRangeQuery(
|
||||
4,
|
||||
mkSet(8, 9)
|
||||
Set.of(8, 9)
|
||||
);
|
||||
|
||||
// not preset, so empty result iter
|
||||
shouldHandleSessionRangeQuery(
|
||||
999,
|
||||
mkSet()
|
||||
Set.of()
|
||||
);
|
||||
|
||||
// Should fail to execute this query on a SessionStore.
|
||||
|
@ -1550,7 +1549,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, T>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, T>> result =
|
||||
|
@ -1606,7 +1605,7 @@ public class IQv2StoreIntegrationTest {
|
|||
|
||||
final UnknownQuery query = new UnknownQuery();
|
||||
final StateQueryRequest<Void> request = inStore(STORE_NAME).withQuery(query);
|
||||
final Set<Integer> partitions = mkSet(0, 1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
|
||||
final StateQueryResult<Void> result =
|
||||
IntegrationTestUtils.iqv2WaitForPartitions(kafkaStreams, request, partitions);
|
||||
|
@ -1645,7 +1644,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<V> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<V> result =
|
||||
|
@ -1679,7 +1678,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<ValueAndTimestamp<V>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<ValueAndTimestamp<V>> result =
|
||||
|
@ -1721,7 +1720,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Integer, V>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
final StateQueryResult<KeyValueIterator<Integer, V>> result =
|
||||
IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
|
@ -1776,7 +1775,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Integer, ValueAndTimestamp<V>>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
final StateQueryResult<KeyValueIterator<Integer, ValueAndTimestamp<V>>> result =
|
||||
IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
|
@ -1831,7 +1830,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<WindowStoreIterator<V>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<WindowStoreIterator<V>> result =
|
||||
|
@ -1881,7 +1880,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, V>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, V>> result =
|
||||
|
@ -1929,7 +1928,7 @@ public class IQv2StoreIntegrationTest {
|
|||
final StateQueryRequest<KeyValueIterator<Windowed<Integer>, V>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
final StateQueryResult<KeyValueIterator<Windowed<Integer>, V>> result =
|
||||
IntegrationTestUtils.iqv2WaitForResult(kafkaStreams, request);
|
||||
|
@ -1970,7 +1969,7 @@ public class IQv2StoreIntegrationTest {
|
|||
public void shouldCollectExecutionInfo() {
|
||||
|
||||
final KeyQuery<Integer, ValueAndTimestamp<Integer>> query = KeyQuery.withKey(1);
|
||||
final Set<Integer> partitions = mkSet(0, 1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
final StateQueryRequest<ValueAndTimestamp<Integer>> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
|
@ -1994,7 +1993,7 @@ public class IQv2StoreIntegrationTest {
|
|||
public void shouldCollectExecutionInfoUnderFailure() {
|
||||
|
||||
final UnknownQuery query = new UnknownQuery();
|
||||
final Set<Integer> partitions = mkSet(0, 1);
|
||||
final Set<Integer> partitions = Set.of(0, 1);
|
||||
final StateQueryRequest<Void> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
|
|
|
@ -61,12 +61,12 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.CyclicBarrier;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
@ -148,7 +148,7 @@ public class LagFetchIntegrationTest {
|
|||
|
||||
IntegrationTestUtils.produceKeyValuesSynchronously(
|
||||
inputTopicName,
|
||||
mkSet(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)),
|
||||
Set.of(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)),
|
||||
TestUtils.producerConfig(
|
||||
CLUSTER.bootstrapServers(),
|
||||
StringSerializer.class,
|
||||
|
@ -211,7 +211,7 @@ public class LagFetchIntegrationTest {
|
|||
// Check the active reports proper lag values.
|
||||
Map<String, Map<Integer, LagInfo>> offsetLagInfoMap = getFirstNonEmptyLagMap(activeStreams);
|
||||
assertThat(offsetLagInfoMap.size(), equalTo(1));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(Set.of(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1));
|
||||
LagInfo lagInfo = offsetLagInfoMap.get(stateStoreName).get(0);
|
||||
assertThat(lagInfo.currentOffsetPosition(), equalTo(5L));
|
||||
|
@ -223,7 +223,7 @@ public class LagFetchIntegrationTest {
|
|||
latchTillStandbyHasPartitionsAssigned.await(60, TimeUnit.SECONDS);
|
||||
offsetLagInfoMap = getFirstNonEmptyLagMap(standbyStreams);
|
||||
assertThat(offsetLagInfoMap.size(), equalTo(1));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(Set.of(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1));
|
||||
lagInfo = offsetLagInfoMap.get(stateStoreName).get(0);
|
||||
assertThat(lagInfo.currentOffsetPosition(), equalTo(0L));
|
||||
|
@ -257,7 +257,7 @@ public class LagFetchIntegrationTest {
|
|||
public void shouldFetchLagsDuringRestoration() throws Exception {
|
||||
IntegrationTestUtils.produceKeyValuesSynchronously(
|
||||
inputTopicName,
|
||||
mkSet(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)),
|
||||
Set.of(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)),
|
||||
TestUtils.producerConfig(
|
||||
CLUSTER.bootstrapServers(),
|
||||
StringSerializer.class,
|
||||
|
@ -296,7 +296,7 @@ public class LagFetchIntegrationTest {
|
|||
TestUtils.waitForCondition(() -> {
|
||||
final Map<String, Map<Integer, LagInfo>> offsetLagInfoMap = streams.allLocalStorePartitionLags();
|
||||
assertThat(offsetLagInfoMap.size(), equalTo(1));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.keySet(), equalTo(Set.of(stateStoreName)));
|
||||
assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1));
|
||||
|
||||
final LagInfo zeroLagInfo = offsetLagInfoMap.get(stateStoreName).get(0);
|
||||
|
|
|
@ -82,7 +82,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.singleton;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.KeyQueryMetadata.NOT_AVAILABLE;
|
||||
import static org.apache.kafka.streams.KeyValue.pair;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.DEFAULT_TIMEOUT;
|
||||
|
@ -291,7 +290,7 @@ public class NamedTopologyIntegrationTest {
|
|||
.filter(t -> t.contains(TOPIC_PREFIX))
|
||||
.filter(t -> t.endsWith("-repartition") || t.endsWith("-changelog") || t.endsWith("-topic"))
|
||||
.collect(Collectors.toSet());
|
||||
assertThat(internalTopics, is(mkSet(
|
||||
assertThat(internalTopics, is(Set.of(
|
||||
countTopicPrefix + "-KSTREAM-AGGREGATE-STATE-STORE-0000000002-repartition",
|
||||
countTopicPrefix + "-KSTREAM-AGGREGATE-STATE-STORE-0000000002-changelog",
|
||||
fkjTopicPrefix + "-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-0000000006-topic",
|
||||
|
@ -385,7 +384,7 @@ public class NamedTopologyIntegrationTest {
|
|||
final Map<String, Map<Integer, LagInfo>> partitionLags2 = streams.allLocalStorePartitionLagsForTopology(TOPOLOGY_2);
|
||||
|
||||
assertThat(partitionLags1.keySet(), equalTo(singleton(topology1Store)));
|
||||
assertThat(partitionLags1.get(topology1Store).keySet(), equalTo(mkSet(0, 1)));
|
||||
assertThat(partitionLags1.get(topology1Store).keySet(), equalTo(Set.of(0, 1)));
|
||||
assertThat(partitionLags2.keySet(), equalTo(singleton(topology2Store)));
|
||||
assertThat(partitionLags2.get(topology2Store).keySet(), equalTo(singleton(0))); // only one copy of the store in topology-2
|
||||
|
||||
|
|
|
@ -85,13 +85,13 @@ import java.util.List;
|
|||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.query.StateQueryRequest.inStore;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -393,7 +393,7 @@ public class PositionRestartIntegrationTest {
|
|||
final StateQueryRequest<?> request =
|
||||
inStore(STORE_NAME)
|
||||
.withQuery(query)
|
||||
.withPartitions(mkSet(0, 1))
|
||||
.withPartitions(Set.of(0, 1))
|
||||
.withPositionBound(PositionBound.at(INPUT_POSITION));
|
||||
|
||||
final StateQueryResult<?> result =
|
||||
|
|
|
@ -102,7 +102,6 @@ import static java.time.Instant.ofEpochMilli;
|
|||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.StoreQueryParameters.fromNameAndType;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.getRunningStreams;
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
|
||||
|
@ -555,7 +554,7 @@ public class QueryableStateIntegrationTest {
|
|||
try {
|
||||
startApplicationAndWaitUntilRunning(streamsList, Duration.ofSeconds(60));
|
||||
|
||||
final Set<String> stores = mkSet(storeName + "-" + streamThree, windowStoreName + "-" + streamThree);
|
||||
final Set<String> stores = Set.of(storeName + "-" + streamThree, windowStoreName + "-" + streamThree);
|
||||
verifyOffsetLagFetch(streamsList, stores, Arrays.asList(4, 4));
|
||||
|
||||
waitUntilAtLeastNumRecordProcessed(outputTopicThree, 1);
|
||||
|
@ -640,7 +639,7 @@ public class QueryableStateIntegrationTest {
|
|||
// create stream threads
|
||||
final String storeName = "word-count-store";
|
||||
final String windowStoreName = "windowed-word-count-store";
|
||||
final Set<String> stores = mkSet(storeName + "-" + streamThree, windowStoreName + "-" + streamThree);
|
||||
final Set<String> stores = Set.of(storeName + "-" + streamThree, windowStoreName + "-" + streamThree);
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
final Properties props = (Properties) streamsConfiguration.clone();
|
||||
props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + i);
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.apache.kafka.common.serialization.Serdes;
|
|||
import org.apache.kafka.common.serialization.StringSerializer;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.KafkaStreams;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
|
@ -54,6 +53,7 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
|
||||
|
@ -226,7 +226,7 @@ public class RocksDBMetricsIntegrationTest {
|
|||
// non-segmented store do not need records with different timestamps
|
||||
IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(
|
||||
STREAM_INPUT_ONE,
|
||||
Utils.mkSet(new KeyValue<>(1, "A"), new KeyValue<>(1, "B"), new KeyValue<>(1, "C")),
|
||||
Set.of(new KeyValue<>(1, "A"), new KeyValue<>(1, "B"), new KeyValue<>(1, "C")),
|
||||
prop,
|
||||
mockTime.milliseconds()
|
||||
);
|
||||
|
|
|
@ -18,7 +18,6 @@ package org.apache.kafka.streams.kstream.internals;
|
|||
|
||||
import org.apache.kafka.clients.consumer.OffsetResetStrategy;
|
||||
import org.apache.kafka.common.utils.Bytes;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsConfig;
|
||||
import org.apache.kafka.streams.errors.TopologyException;
|
||||
|
@ -212,7 +211,7 @@ public class InternalStreamsBuilderTest {
|
|||
final List<StateStore> stateStores = topology.globalStateStores();
|
||||
final Set<String> sourceTopics = topology.sourceTopics();
|
||||
|
||||
assertEquals(Utils.mkSet("table", "table2"), sourceTopics);
|
||||
assertEquals(Set.of("table", "table2"), sourceTopics);
|
||||
assertEquals(2, stateStores.size());
|
||||
}
|
||||
|
||||
|
|
|
@ -79,7 +79,6 @@ import static java.time.Duration.ofMillis;
|
|||
import static java.util.Arrays.asList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.hasItem;
|
||||
import static org.hamcrest.CoreMatchers.hasItems;
|
||||
|
@ -155,63 +154,63 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
final Map<Long, Set<ValueAndTimestamp<String>>> expected = new HashMap<>();
|
||||
|
||||
if (emitFinal) {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2", 10L)
|
||||
));
|
||||
expected.put(4L, mkSet(
|
||||
expected.put(4L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3", 14L)
|
||||
));
|
||||
expected.put(5L, mkSet(
|
||||
expected.put(5L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4", 15L)
|
||||
));
|
||||
expected.put(10L, mkSet(
|
||||
expected.put(10L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5", 20L)
|
||||
));
|
||||
expected.put(11L, mkSet(
|
||||
expected.put(11L, Set.of(
|
||||
ValueAndTimestamp.make("0+3+4+5", 20L)
|
||||
));
|
||||
expected.put(12L, mkSet(
|
||||
expected.put(12L, Set.of(
|
||||
ValueAndTimestamp.make("0+3+4+5+6", 22L)
|
||||
));
|
||||
} else {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1", 10L),
|
||||
ValueAndTimestamp.make("0+1+2", 10L)
|
||||
));
|
||||
expected.put(4L, mkSet(
|
||||
expected.put(4L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3", 14L)
|
||||
));
|
||||
expected.put(5L, mkSet(
|
||||
expected.put(5L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4", 15L)
|
||||
));
|
||||
expected.put(10L, mkSet(
|
||||
expected.put(10L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5", 20L)
|
||||
));
|
||||
expected.put(11L, mkSet(
|
||||
expected.put(11L, Set.of(
|
||||
ValueAndTimestamp.make("0+3", 14L),
|
||||
ValueAndTimestamp.make("0+3+4", 15L),
|
||||
ValueAndTimestamp.make("0+3+4+5", 20L)
|
||||
));
|
||||
expected.put(12L, mkSet(
|
||||
expected.put(12L, Set.of(
|
||||
ValueAndTimestamp.make("0+3+4+5+6", 22L)
|
||||
));
|
||||
expected.put(15L, mkSet(
|
||||
expected.put(15L, Set.of(
|
||||
ValueAndTimestamp.make("0+4", 15L),
|
||||
ValueAndTimestamp.make("0+4+5", 20L),
|
||||
ValueAndTimestamp.make("0+4+5+6", 22L)
|
||||
));
|
||||
expected.put(16L, mkSet(
|
||||
expected.put(16L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 20L),
|
||||
ValueAndTimestamp.make("0+5+6", 22L)
|
||||
));
|
||||
expected.put(20L, mkSet(
|
||||
expected.put(20L, Set.of(
|
||||
ValueAndTimestamp.make("0+5+6+7", 30L)
|
||||
));
|
||||
expected.put(21L, mkSet(
|
||||
expected.put(21L, Set.of(
|
||||
ValueAndTimestamp.make("0+6", 22L),
|
||||
ValueAndTimestamp.make("0+6+7", 30L)
|
||||
));
|
||||
expected.put(23L, mkSet(
|
||||
expected.put(23L, Set.of(
|
||||
ValueAndTimestamp.make("0+7", 30L)
|
||||
));
|
||||
}
|
||||
|
@ -254,34 +253,34 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
final Map<Long, Set<ValueAndTimestamp<String>>> expected = new HashMap<>();
|
||||
|
||||
if (emitFinal) {
|
||||
expected.put(0L, mkSet(ValueAndTimestamp.make("1", 10L)));
|
||||
expected.put(4L, mkSet(ValueAndTimestamp.make("1+2", 14L)));
|
||||
expected.put(5L, mkSet(ValueAndTimestamp.make("1+2+3", 15L)));
|
||||
expected.put(11L, mkSet(ValueAndTimestamp.make("2+3", 15L)));
|
||||
expected.put(12L, mkSet(ValueAndTimestamp.make("2+3+4", 22L)));
|
||||
expected.put(0L, Set.of(ValueAndTimestamp.make("1", 10L)));
|
||||
expected.put(4L, Set.of(ValueAndTimestamp.make("1+2", 14L)));
|
||||
expected.put(5L, Set.of(ValueAndTimestamp.make("1+2+3", 15L)));
|
||||
expected.put(11L, Set.of(ValueAndTimestamp.make("2+3", 15L)));
|
||||
expected.put(12L, Set.of(ValueAndTimestamp.make("2+3+4", 22L)));
|
||||
} else {
|
||||
expected.put(0L, mkSet(ValueAndTimestamp.make("1", 10L)));
|
||||
expected.put(4L, mkSet(ValueAndTimestamp.make("1+2", 14L)));
|
||||
expected.put(5L, mkSet(ValueAndTimestamp.make("1+2+3", 15L)));
|
||||
expected.put(11L, mkSet(
|
||||
expected.put(0L, Set.of(ValueAndTimestamp.make("1", 10L)));
|
||||
expected.put(4L, Set.of(ValueAndTimestamp.make("1+2", 14L)));
|
||||
expected.put(5L, Set.of(ValueAndTimestamp.make("1+2+3", 15L)));
|
||||
expected.put(11L, Set.of(
|
||||
ValueAndTimestamp.make("2", 14L),
|
||||
ValueAndTimestamp.make("2+3", 15L)
|
||||
));
|
||||
expected.put(12L, mkSet(ValueAndTimestamp.make("2+3+4", 22L)));
|
||||
expected.put(15L, mkSet(
|
||||
expected.put(12L, Set.of(ValueAndTimestamp.make("2+3+4", 22L)));
|
||||
expected.put(15L, Set.of(
|
||||
ValueAndTimestamp.make("3", 15L),
|
||||
ValueAndTimestamp.make("3+4", 22L)
|
||||
));
|
||||
expected.put(16L, mkSet(
|
||||
expected.put(16L, Set.of(
|
||||
ValueAndTimestamp.make("4", 22L),
|
||||
ValueAndTimestamp.make("4+5", 26L)
|
||||
));
|
||||
expected.put(20L, mkSet(ValueAndTimestamp.make("4+5+6", 30L)));
|
||||
expected.put(23L, mkSet(
|
||||
expected.put(20L, Set.of(ValueAndTimestamp.make("4+5+6", 30L)));
|
||||
expected.put(23L, Set.of(
|
||||
ValueAndTimestamp.make("5", 26L),
|
||||
ValueAndTimestamp.make("5+6", 30L)
|
||||
));
|
||||
expected.put(27L, mkSet(ValueAndTimestamp.make("6", 30L)));
|
||||
expected.put(27L, Set.of(ValueAndTimestamp.make("6", 30L)));
|
||||
}
|
||||
assertEquals(expected, actual);
|
||||
}
|
||||
|
@ -697,38 +696,38 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
|
||||
if (emitFinal) {
|
||||
expected.put(0L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(1L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+4+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(4L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(6L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+3+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(7L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(11L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L)
|
||||
)
|
||||
);
|
||||
} else {
|
||||
expected.put(0L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+1", 0L),
|
||||
ValueAndTimestamp.make("0+1+2", 5L),
|
||||
ValueAndTimestamp.make("0+1+2+3", 6L),
|
||||
|
@ -738,7 +737,7 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
)
|
||||
);
|
||||
expected.put(1L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+2", 5L),
|
||||
ValueAndTimestamp.make("0+2+3", 6L),
|
||||
ValueAndTimestamp.make("0+2+3+4", 6L),
|
||||
|
@ -747,32 +746,32 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
)
|
||||
);
|
||||
expected.put(4L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+2+3", 6L),
|
||||
ValueAndTimestamp.make("0+2+3+5", 13L),
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(6L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+3", 6L),
|
||||
ValueAndTimestamp.make("0+3+5", 13L),
|
||||
ValueAndTimestamp.make("0+3+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(7L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L),
|
||||
ValueAndTimestamp.make("0+5+6", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(11L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L)
|
||||
)
|
||||
);
|
||||
expected.put(20L,
|
||||
mkSet(
|
||||
Set.of(
|
||||
ValueAndTimestamp.make("0+7", 70L)
|
||||
)
|
||||
);
|
||||
|
@ -829,14 +828,14 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
|
||||
final Map<Long, Set<ValueAndTimestamp<String>>> expected = new HashMap<>();
|
||||
if (emitFinal) {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6+7", 4L)
|
||||
));
|
||||
expected.put(1L, mkSet(
|
||||
expected.put(1L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 4L)
|
||||
));
|
||||
} else {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1", 0L),
|
||||
ValueAndTimestamp.make("0+1+2", 2L),
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5", 4L),
|
||||
|
@ -845,20 +844,20 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
ValueAndTimestamp.make("0+1+2+3+4", 4L),
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6+7", 4L)
|
||||
));
|
||||
expected.put(1L, mkSet(
|
||||
expected.put(1L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 4L),
|
||||
ValueAndTimestamp.make("0+2", 2L),
|
||||
ValueAndTimestamp.make("0+2+3", 4L),
|
||||
ValueAndTimestamp.make("0+2+3+5", 4L)
|
||||
));
|
||||
expected.put(2L, mkSet(
|
||||
expected.put(2L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+5+6+8", 7)
|
||||
));
|
||||
expected.put(3L, mkSet(
|
||||
expected.put(3L, Set.of(
|
||||
ValueAndTimestamp.make("0+3", 4L),
|
||||
ValueAndTimestamp.make("0+3+8", 7L)
|
||||
));
|
||||
expected.put(5L, mkSet(
|
||||
expected.put(5L, Set.of(
|
||||
ValueAndTimestamp.make("0+8", 7)
|
||||
));
|
||||
}
|
||||
|
@ -1061,26 +1060,26 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
final Map<Long, Set<ValueAndTimestamp<String>>> expected = new HashMap<>();
|
||||
|
||||
if (emitFinal) {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6", 13L)
|
||||
));
|
||||
expected.put(1L, mkSet(
|
||||
expected.put(1L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+4+5+6", 13L)
|
||||
));
|
||||
expected.put(4L, mkSet(
|
||||
expected.put(4L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 13L)
|
||||
));
|
||||
expected.put(6L, mkSet(
|
||||
expected.put(6L, Set.of(
|
||||
ValueAndTimestamp.make("0+3+5+6", 13L)
|
||||
));
|
||||
expected.put(7L, mkSet(
|
||||
expected.put(7L, Set.of(
|
||||
ValueAndTimestamp.make("0+5+6", 13L)
|
||||
));
|
||||
expected.put(11L, mkSet(
|
||||
expected.put(11L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L)
|
||||
));
|
||||
} else {
|
||||
expected.put(0L, mkSet(
|
||||
expected.put(0L, Set.of(
|
||||
ValueAndTimestamp.make("0+1", 0L),
|
||||
ValueAndTimestamp.make("0+1+2", 5L),
|
||||
ValueAndTimestamp.make("0+1+2+3", 6L),
|
||||
|
@ -1088,31 +1087,31 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
ValueAndTimestamp.make("0+1+2+3+4+5", 13L),
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6", 13L)
|
||||
));
|
||||
expected.put(1L, mkSet(
|
||||
expected.put(1L, Set.of(
|
||||
ValueAndTimestamp.make("0+2", 5L),
|
||||
ValueAndTimestamp.make("0+2+3", 6L),
|
||||
ValueAndTimestamp.make("0+2+3+4", 6L),
|
||||
ValueAndTimestamp.make("0+2+3+4+5", 13L),
|
||||
ValueAndTimestamp.make("0+2+3+4+5+6", 13L)
|
||||
));
|
||||
expected.put(4L, mkSet(
|
||||
expected.put(4L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3", 6L),
|
||||
ValueAndTimestamp.make("0+2+3+5", 13L),
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 13L)
|
||||
));
|
||||
expected.put(6L, mkSet(
|
||||
expected.put(6L, Set.of(
|
||||
ValueAndTimestamp.make("0+3", 6L),
|
||||
ValueAndTimestamp.make("0+3+5", 13L),
|
||||
ValueAndTimestamp.make("0+3+5+6", 13L)
|
||||
));
|
||||
expected.put(7L, mkSet(
|
||||
expected.put(7L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L),
|
||||
ValueAndTimestamp.make("0+5+6", 13L)
|
||||
));
|
||||
expected.put(11L, mkSet(
|
||||
expected.put(11L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 13L)
|
||||
));
|
||||
expected.put(20L, mkSet(
|
||||
expected.put(20L, Set.of(
|
||||
ValueAndTimestamp.make("0+6", 70L)
|
||||
));
|
||||
}
|
||||
|
@ -1159,52 +1158,52 @@ public class KStreamSlidingWindowAggregateTest {
|
|||
final Map<Long, Set<ValueAndTimestamp<String>>> expected = new HashMap<>();
|
||||
|
||||
if (emitFinal) {
|
||||
expected.put(50L, mkSet(
|
||||
expected.put(50L, Set.of(
|
||||
ValueAndTimestamp.make("0+1", 100L)
|
||||
));
|
||||
expected.put(55L, mkSet(
|
||||
expected.put(55L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2", 105L)
|
||||
));
|
||||
expected.put(56L, mkSet(
|
||||
expected.put(56L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4", 106L)
|
||||
));
|
||||
} else {
|
||||
expected.put(50L, mkSet(
|
||||
expected.put(50L, Set.of(
|
||||
ValueAndTimestamp.make("0+1", 100L)
|
||||
));
|
||||
expected.put(55L, mkSet(
|
||||
expected.put(55L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2", 105L)
|
||||
));
|
||||
expected.put(56L, mkSet(
|
||||
expected.put(56L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3", 106L),
|
||||
ValueAndTimestamp.make("0+1+2+3+4", 106L)
|
||||
));
|
||||
expected.put(63L, mkSet(
|
||||
expected.put(63L, Set.of(
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5", 113L),
|
||||
ValueAndTimestamp.make("0+1+2+3+4+5+6", 113L)
|
||||
));
|
||||
expected.put(101L, mkSet(
|
||||
expected.put(101L, Set.of(
|
||||
ValueAndTimestamp.make("0+2", 105L),
|
||||
ValueAndTimestamp.make("0+2+3", 106L),
|
||||
ValueAndTimestamp.make("0+2+3+4", 106L),
|
||||
ValueAndTimestamp.make("0+2+3+4+5", 113L),
|
||||
ValueAndTimestamp.make("0+2+3+4+5+6", 113L)
|
||||
));
|
||||
expected.put(104L, mkSet(
|
||||
expected.put(104L, Set.of(
|
||||
ValueAndTimestamp.make("0+2+3", 106L),
|
||||
ValueAndTimestamp.make("0+2+3+5", 113L),
|
||||
ValueAndTimestamp.make("0+2+3+5+6", 113L)
|
||||
));
|
||||
expected.put(106L, mkSet(
|
||||
expected.put(106L, Set.of(
|
||||
ValueAndTimestamp.make("0+3", 106L),
|
||||
ValueAndTimestamp.make("0+3+5", 113L),
|
||||
ValueAndTimestamp.make("0+3+5+6", 113L)
|
||||
));
|
||||
expected.put(107L, mkSet(
|
||||
expected.put(107L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 113L),
|
||||
ValueAndTimestamp.make("0+5+6", 113L)
|
||||
));
|
||||
expected.put(111L, mkSet(
|
||||
expected.put(111L, Set.of(
|
||||
ValueAndTimestamp.make("0+5", 113L)
|
||||
));
|
||||
}
|
||||
|
|
|
@ -41,11 +41,11 @@ import org.junit.jupiter.api.Test;
|
|||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
|
@ -214,7 +214,7 @@ public class KTableKTableForeignKeyJoinScenarioTest {
|
|||
}
|
||||
// verifying primarily that no extra pseudo-topics were used, but it's nice to also verify the rest of the
|
||||
// topics our serdes serialize data for
|
||||
assertThat(serdeScope.registeredTopics(), is(mkSet(
|
||||
assertThat(serdeScope.registeredTopics(), is(Set.of(
|
||||
// expected pseudo-topics
|
||||
applicationId + "-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-0000000006-topic-fk--key",
|
||||
applicationId + "-KTABLE-FK-JOIN-SUBSCRIPTION-REGISTRATION-0000000006-topic-pk--key",
|
||||
|
|
|
@ -45,13 +45,13 @@ import java.io.File;
|
|||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static java.util.Collections.emptySet;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
@ -274,7 +274,7 @@ public class ActiveTaskCreatorTest {
|
|||
mkEntry(task01, Collections.singleton(new TopicPartition("topic", 1)))
|
||||
)
|
||||
).stream().map(Task::id).collect(Collectors.toSet()),
|
||||
equalTo(mkSet(task00, task01))
|
||||
equalTo(Set.of(task00, task01))
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -33,7 +33,6 @@ import java.util.Set;
|
|||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
|
@ -55,26 +54,26 @@ public class ChangelogTopicsTest {
|
|||
new UnwindowedUnversionedChangelogTopicConfig(CHANGELOG_TOPIC_NAME1, TOPIC_CONFIG);
|
||||
|
||||
private static final TopicsInfo TOPICS_INFO1 = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME),
|
||||
mkSet(SOURCE_TOPIC_NAME),
|
||||
Set.of(SINK_TOPIC_NAME),
|
||||
Set.of(SOURCE_TOPIC_NAME),
|
||||
mkMap(mkEntry(REPARTITION_TOPIC_NAME, REPARTITION_TOPIC_CONFIG)),
|
||||
mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG))
|
||||
);
|
||||
private static final TopicsInfo TOPICS_INFO2 = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME),
|
||||
mkSet(SOURCE_TOPIC_NAME),
|
||||
Set.of(SINK_TOPIC_NAME),
|
||||
Set.of(SOURCE_TOPIC_NAME),
|
||||
mkMap(mkEntry(REPARTITION_TOPIC_NAME, REPARTITION_TOPIC_CONFIG)),
|
||||
mkMap()
|
||||
);
|
||||
private static final TopicsInfo TOPICS_INFO3 = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME),
|
||||
mkSet(SOURCE_TOPIC_NAME),
|
||||
Set.of(SINK_TOPIC_NAME),
|
||||
Set.of(SOURCE_TOPIC_NAME),
|
||||
mkMap(mkEntry(REPARTITION_TOPIC_NAME, REPARTITION_TOPIC_CONFIG)),
|
||||
mkMap(mkEntry(SOURCE_TOPIC_NAME, CHANGELOG_TOPIC_CONFIG))
|
||||
);
|
||||
private static final TopicsInfo TOPICS_INFO4 = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME),
|
||||
mkSet(SOURCE_TOPIC_NAME),
|
||||
Set.of(SINK_TOPIC_NAME),
|
||||
Set.of(SOURCE_TOPIC_NAME),
|
||||
mkMap(mkEntry(REPARTITION_TOPIC_NAME, REPARTITION_TOPIC_CONFIG)),
|
||||
mkMap(mkEntry(SOURCE_TOPIC_NAME, null), mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG))
|
||||
);
|
||||
|
@ -88,7 +87,7 @@ public class ChangelogTopicsTest {
|
|||
public void shouldNotContainChangelogsForStatelessTasks() {
|
||||
when(internalTopicManager.makeReady(Collections.emptyMap())).thenReturn(Collections.emptySet());
|
||||
final Map<Subtopology, TopicsInfo> topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO2));
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, mkSet(TASK_0_0, TASK_0_1, TASK_0_2)));
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, Set.of(TASK_0_0, TASK_0_1, TASK_0_2)));
|
||||
|
||||
final ChangelogTopics changelogTopics =
|
||||
new ChangelogTopics(internalTopicManager, topicGroups, tasksForTopicGroup, "[test] ");
|
||||
|
@ -104,9 +103,9 @@ public class ChangelogTopicsTest {
|
|||
@Test
|
||||
public void shouldNotContainAnyPreExistingChangelogsIfChangelogIsNewlyCreated() {
|
||||
when(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG))))
|
||||
.thenReturn(mkSet(CHANGELOG_TOPIC_NAME1));
|
||||
.thenReturn(Set.of(CHANGELOG_TOPIC_NAME1));
|
||||
final Map<Subtopology, TopicsInfo> topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1));
|
||||
final Set<TaskId> tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> tasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks));
|
||||
|
||||
final ChangelogTopics changelogTopics =
|
||||
|
@ -126,7 +125,7 @@ public class ChangelogTopicsTest {
|
|||
when(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG))))
|
||||
.thenReturn(Collections.emptySet());
|
||||
final Map<Subtopology, TopicsInfo> topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1));
|
||||
final Set<TaskId> tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> tasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks));
|
||||
|
||||
final ChangelogTopics changelogTopics =
|
||||
|
@ -137,13 +136,13 @@ public class ChangelogTopicsTest {
|
|||
final TopicPartition changelogPartition0 = new TopicPartition(CHANGELOG_TOPIC_NAME1, 0);
|
||||
final TopicPartition changelogPartition1 = new TopicPartition(CHANGELOG_TOPIC_NAME1, 1);
|
||||
final TopicPartition changelogPartition2 = new TopicPartition(CHANGELOG_TOPIC_NAME1, 2);
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(mkSet(changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(mkSet(changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(mkSet(changelogPartition2)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(Set.of(changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(Set.of(changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(Set.of(changelogPartition2)));
|
||||
assertThat(changelogTopics.preExistingSourceTopicBasedPartitions(), is(Collections.emptySet()));
|
||||
assertThat(
|
||||
changelogTopics.preExistingNonSourceTopicBasedPartitions(),
|
||||
is(mkSet(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
is(Set.of(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -151,7 +150,7 @@ public class ChangelogTopicsTest {
|
|||
public void shouldOnlyContainPreExistingSourceBasedChangelogs() {
|
||||
when(internalTopicManager.makeReady(Collections.emptyMap())).thenReturn(Collections.emptySet());
|
||||
final Map<Subtopology, TopicsInfo> topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO3));
|
||||
final Set<TaskId> tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> tasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks));
|
||||
|
||||
final ChangelogTopics changelogTopics =
|
||||
|
@ -161,12 +160,12 @@ public class ChangelogTopicsTest {
|
|||
final TopicPartition changelogPartition0 = new TopicPartition(SOURCE_TOPIC_NAME, 0);
|
||||
final TopicPartition changelogPartition1 = new TopicPartition(SOURCE_TOPIC_NAME, 1);
|
||||
final TopicPartition changelogPartition2 = new TopicPartition(SOURCE_TOPIC_NAME, 2);
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(mkSet(changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(mkSet(changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(mkSet(changelogPartition2)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(Set.of(changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(Set.of(changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(Set.of(changelogPartition2)));
|
||||
assertThat(
|
||||
changelogTopics.preExistingSourceTopicBasedPartitions(),
|
||||
is(mkSet(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
is(Set.of(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
);
|
||||
assertThat(changelogTopics.preExistingNonSourceTopicBasedPartitions(), is(Collections.emptySet()));
|
||||
}
|
||||
|
@ -176,7 +175,7 @@ public class ChangelogTopicsTest {
|
|||
when(internalTopicManager.makeReady(mkMap(mkEntry(CHANGELOG_TOPIC_NAME1, CHANGELOG_TOPIC_CONFIG))))
|
||||
.thenReturn(Collections.emptySet());
|
||||
final Map<Subtopology, TopicsInfo> topicGroups = mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO4));
|
||||
final Set<TaskId> tasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> tasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Map<Subtopology, Set<TaskId>> tasksForTopicGroup = mkMap(mkEntry(SUBTOPOLOGY_0, tasks));
|
||||
|
||||
final ChangelogTopics changelogTopics =
|
||||
|
@ -190,16 +189,16 @@ public class ChangelogTopicsTest {
|
|||
final TopicPartition sourcePartition0 = new TopicPartition(SOURCE_TOPIC_NAME, 0);
|
||||
final TopicPartition sourcePartition1 = new TopicPartition(SOURCE_TOPIC_NAME, 1);
|
||||
final TopicPartition sourcePartition2 = new TopicPartition(SOURCE_TOPIC_NAME, 2);
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(mkSet(sourcePartition0, changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(mkSet(sourcePartition1, changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(mkSet(sourcePartition2, changelogPartition2)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_0), is(Set.of(sourcePartition0, changelogPartition0)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_1), is(Set.of(sourcePartition1, changelogPartition1)));
|
||||
assertThat(changelogTopics.preExistingPartitionsFor(TASK_0_2), is(Set.of(sourcePartition2, changelogPartition2)));
|
||||
assertThat(
|
||||
changelogTopics.preExistingSourceTopicBasedPartitions(),
|
||||
is(mkSet(sourcePartition0, sourcePartition1, sourcePartition2))
|
||||
is(Set.of(sourcePartition0, sourcePartition1, sourcePartition2))
|
||||
);
|
||||
assertThat(
|
||||
changelogTopics.preExistingNonSourceTopicBasedPartitions(),
|
||||
is(mkSet(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
is(Set.of(changelogPartition0, changelogPartition1, changelogPartition2))
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -46,7 +46,6 @@ import java.util.concurrent.ExecutionException;
|
|||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.emptySet;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.consumerRecordSizeInBytes;
|
||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchCommittedOffsets;
|
||||
import static org.apache.kafka.streams.processor.internals.ClientUtils.fetchEndOffsets;
|
||||
|
@ -107,7 +106,7 @@ public class ClientUtilsTest {
|
|||
HEADERS_BYTES +
|
||||
RECORD_METADATA_BYTES;
|
||||
|
||||
private static final Set<TopicPartition> PARTITIONS = mkSet(
|
||||
private static final Set<TopicPartition> PARTITIONS = Set.of(
|
||||
new TopicPartition(TOPIC, 1),
|
||||
new TopicPartition(TOPIC, 2)
|
||||
);
|
||||
|
|
|
@ -30,6 +30,7 @@ import java.util.Collections;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
@ -68,7 +69,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
@Test
|
||||
public void shouldThrowTopologyBuilderExceptionIfPartitionCountsForCoPartitionedTopicsDontMatch() {
|
||||
partitions.remove(new TopicPartition("second", 0));
|
||||
assertThrows(TopologyException.class, () -> validator.enforce(Utils.mkSet("first", "second"),
|
||||
assertThrows(TopologyException.class, () -> validator.enforce(Set.of("first", "second"),
|
||||
Collections.emptyMap(),
|
||||
cluster.withPartitions(partitions)));
|
||||
}
|
||||
|
@ -78,7 +79,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
public void shouldEnforceCopartitioningOnRepartitionTopics() {
|
||||
final InternalTopicConfig config = createTopicConfig("repartitioned", 10);
|
||||
|
||||
validator.enforce(Utils.mkSet("first", "second", config.name()),
|
||||
validator.enforce(Set.of("first", "second", config.name()),
|
||||
Collections.singletonMap(config.name(), config),
|
||||
cluster.withPartitions(partitions));
|
||||
|
||||
|
@ -97,7 +98,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
repartitionTopicConfig.put(two.name(), two);
|
||||
repartitionTopicConfig.put(three.name(), three);
|
||||
|
||||
validator.enforce(Utils.mkSet(one.name(),
|
||||
validator.enforce(Set.of(one.name(),
|
||||
two.name(),
|
||||
three.name()),
|
||||
repartitionTopicConfig,
|
||||
|
@ -116,7 +117,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
|
||||
final TopologyException ex = assertThrows(
|
||||
TopologyException.class,
|
||||
() -> validator.enforce(Utils.mkSet(topic1.name(), topic2.name()),
|
||||
() -> validator.enforce(Set.of(topic1.name(), topic2.name()),
|
||||
Utils.mkMap(Utils.mkEntry(topic1.name(), topic1),
|
||||
Utils.mkEntry(topic2.name(), topic2)),
|
||||
cluster.withPartitions(partitions))
|
||||
|
@ -137,7 +138,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
final InternalTopicConfig topic1 = createRepartitionTopicConfigWithEnforcedNumberOfPartitions("repartitioned-1", 10);
|
||||
final InternalTopicConfig topic2 = createRepartitionTopicConfigWithEnforcedNumberOfPartitions("repartitioned-2", 10);
|
||||
|
||||
validator.enforce(Utils.mkSet(topic1.name(), topic2.name()),
|
||||
validator.enforce(Set.of(topic1.name(), topic2.name()),
|
||||
Utils.mkMap(Utils.mkEntry(topic1.name(), topic1),
|
||||
Utils.mkEntry(topic2.name(), topic2)),
|
||||
cluster.withPartitions(partitions));
|
||||
|
@ -152,7 +153,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
|
||||
final TopologyException ex = assertThrows(
|
||||
TopologyException.class,
|
||||
() -> validator.enforce(Utils.mkSet(topic1.name(), "second"),
|
||||
() -> validator.enforce(Set.of(topic1.name(), "second"),
|
||||
Utils.mkMap(Utils.mkEntry(topic1.name(), topic1)),
|
||||
cluster.withPartitions(partitions))
|
||||
);
|
||||
|
@ -167,7 +168,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
public void shouldNotThrowAnExceptionWhenNumberOfPartitionsOfNonRepartitionTopicAndRepartitionTopicWithEnforcedNumOfPartitionsMatch() {
|
||||
final InternalTopicConfig topic1 = createRepartitionTopicConfigWithEnforcedNumberOfPartitions("repartitioned-1", 2);
|
||||
|
||||
validator.enforce(Utils.mkSet(topic1.name(), "second"),
|
||||
validator.enforce(Set.of(topic1.name(), "second"),
|
||||
Utils.mkMap(Utils.mkEntry(topic1.name(), topic1)),
|
||||
cluster.withPartitions(partitions));
|
||||
|
||||
|
@ -180,7 +181,7 @@ public class CopartitionedTopicsEnforcerTest {
|
|||
final InternalTopicConfig topic2 = createTopicConfig("repartitioned-2", 5);
|
||||
final InternalTopicConfig topic3 = createRepartitionTopicConfigWithEnforcedNumberOfPartitions("repartitioned-3", 2);
|
||||
|
||||
validator.enforce(Utils.mkSet(topic1.name(), topic2.name()),
|
||||
validator.enforce(Set.of(topic1.name(), topic2.name()),
|
||||
Utils.mkMap(Utils.mkEntry(topic1.name(), topic1),
|
||||
Utils.mkEntry(topic2.name(), topic2),
|
||||
Utils.mkEntry(topic3.name(), topic3)),
|
||||
|
|
|
@ -53,7 +53,6 @@ import java.util.stream.Collectors;
|
|||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.StreamsConfig.producerPrefix;
|
||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.standbyTask;
|
||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statefulTask;
|
||||
|
@ -131,11 +130,11 @@ class DefaultStateUpdaterTest {
|
|||
@Test
|
||||
public void shouldShutdownStateUpdater() throws Exception {
|
||||
final StreamTask statelessTask = statelessTask(TASK_0_0).inState(State.RESTORING).build();
|
||||
final StreamTask restoredStatefulTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask failedStatefulTask = statefulTask(TASK_1_1, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(mkSet(TOPIC_PARTITION_B_0));
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(mkSet(TASK_1_1));
|
||||
final StreamTask restoredStatefulTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask failedStatefulTask = statefulTask(TASK_1_1, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Set.of(TOPIC_PARTITION_B_0));
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(Set.of(TASK_1_1));
|
||||
doThrow(taskCorruptedException).when(changelogReader).restore(mkMap(
|
||||
mkEntry(TASK_1_1, failedStatefulTask),
|
||||
mkEntry(TASK_0_2, standbyTask)
|
||||
|
@ -161,8 +160,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldShutdownStateUpdaterWithPausedTasks() throws Exception {
|
||||
final StreamTask statefulTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask statefulTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
when(topologyMetadata.isPaused(null)).thenReturn(true);
|
||||
stateUpdater.add(statefulTask);
|
||||
stateUpdater.add(standbyTask);
|
||||
|
@ -196,8 +195,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRestartedWithNonEmptyRestoredTasks() throws Exception {
|
||||
final StreamTask restoredTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_A_1));
|
||||
final StreamTask restoredTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_A_1));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(restoredTask);
|
||||
|
@ -213,8 +212,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRestartedWithNonEmptyFailedTasks() throws Exception {
|
||||
final StreamTask failedTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(mkSet(TASK_0_0));
|
||||
final StreamTask failedTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(Set.of(TASK_0_0));
|
||||
doThrow(taskCorruptedException).when(changelogReader).restore(mkMap(mkEntry(TASK_0_0, failedTask)));
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(failedTask);
|
||||
|
@ -235,7 +234,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfStatefulTaskNotInStateRestoring() {
|
||||
shouldThrowIfActiveTaskNotInStateRestoring(statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).build());
|
||||
shouldThrowIfActiveTaskNotInStateRestoring(statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).build());
|
||||
}
|
||||
|
||||
private void shouldThrowIfActiveTaskNotInStateRestoring(final StreamTask task) {
|
||||
|
@ -244,7 +243,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfStandbyTaskNotInStateRunning() {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).build();
|
||||
shouldThrowIfTaskNotInGivenState(task, State.RUNNING);
|
||||
}
|
||||
|
||||
|
@ -259,29 +258,29 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfAddingActiveTasksWithSameId() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldThrowIfAddingTasksWithSameId(task1, task2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowIfAddingStandbyTasksWithSameId() throws Exception {
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
shouldThrowIfAddingTasksWithSameId(task1, task2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowIfAddingActiveAndStandbyTaskWithSameId() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
shouldThrowIfAddingTasksWithSameId(task1, task2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldThrowIfAddingStandbyAndActiveTaskWithSameId() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
shouldThrowIfAddingTasksWithSameId(task2, task1);
|
||||
}
|
||||
|
||||
|
@ -324,11 +323,11 @@ class DefaultStateUpdaterTest {
|
|||
@Test
|
||||
public void shouldRestoreSingleActiveStatefulTask() throws Exception {
|
||||
final StreamTask task =
|
||||
statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted())
|
||||
.thenReturn(false)
|
||||
.thenReturn(false)
|
||||
|
@ -351,17 +350,17 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRestoreMultipleActiveStatefulTasks() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final AtomicBoolean allChangelogCompleted = new AtomicBoolean(false);
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_C_0))
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_C_0, TOPIC_PARTITION_A_0))
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_C_0))
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_C_0, TOPIC_PARTITION_A_0))
|
||||
.thenAnswer(invocation -> {
|
||||
allChangelogCompleted.set(true);
|
||||
return mkSet(TOPIC_PARTITION_C_0, TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0);
|
||||
return Set.of(TOPIC_PARTITION_C_0, TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0);
|
||||
});
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(allChangelogCompleted.get());
|
||||
stateUpdater.start();
|
||||
|
@ -388,7 +387,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfTaskAdded() {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
stateUpdater.add(task);
|
||||
|
||||
|
@ -397,7 +396,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfTaskUpdating() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet());
|
||||
|
@ -415,10 +414,10 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfTaskRestored() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted())
|
||||
.thenReturn(true);
|
||||
stateUpdater.start();
|
||||
|
@ -433,7 +432,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnFalseForRestoreActiveTasksIfTaskRemoved() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet());
|
||||
|
@ -452,13 +451,13 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfTaskFailed() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted())
|
||||
.thenReturn(false);
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(mkSet(task.id()));
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(Set.of(task.id()));
|
||||
doThrow(taskCorruptedException).when(changelogReader).restore(mkMap(mkEntry(TASK_0_0, task)));
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(task);
|
||||
|
@ -472,7 +471,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfTaskPaused() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet());
|
||||
|
@ -492,10 +491,10 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnFalseForRestoreActiveTasksIfTaskRemovedFromStateUpdater() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted())
|
||||
.thenReturn(true);
|
||||
stateUpdater.start();
|
||||
|
@ -511,7 +510,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnTrueForRestoreActiveTasksIfStandbyTask() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet());
|
||||
|
@ -549,16 +548,16 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldUpdateSingleStandbyTask() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0))
|
||||
.inState(State.RUNNING).build();
|
||||
shouldUpdateStandbyTasks(task);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldUpdateMultipleStandbyTasks() throws Exception {
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
shouldUpdateStandbyTasks(task1, task2, task3);
|
||||
}
|
||||
|
||||
|
@ -585,14 +584,14 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRestoreActiveStatefulTasksAndUpdateStandbyTasks() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
||||
|
@ -618,13 +617,13 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRestoreActiveStatefulTaskThenUpdateStandbyTaskAndAgainRestoreActiveStatefulTask() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs())
|
||||
.thenReturn(Collections.emptySet())
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(mkSet(TOPIC_PARTITION_B_0));
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_A_0))
|
||||
.thenReturn(Set.of(TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
||||
|
@ -648,11 +647,11 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldUpdateStandbyTaskAfterAllActiveStatefulTasksFailed() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final TaskCorruptedException taskCorruptedException =
|
||||
new TaskCorruptedException(mkSet(activeTask1.id(), activeTask2.id()));
|
||||
new TaskCorruptedException(Set.of(activeTask1.id(), activeTask2.id()));
|
||||
final Map<TaskId, Task> updatingTasks1 = mkMap(
|
||||
mkEntry(activeTask1.id(), activeTask1),
|
||||
mkEntry(activeTask2.id(), activeTask2),
|
||||
|
@ -676,13 +675,13 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotTransitToStandbyAgainAfterStandbyTaskFailed() throws Exception {
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task1 = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final Map<TaskId, Task> updatingTasks = mkMap(
|
||||
mkEntry(task1.id(), task1),
|
||||
mkEntry(task2.id(), task2)
|
||||
);
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(mkSet(task1.id()));
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(Set.of(task1.id()));
|
||||
final ExceptionAndTask expectedExceptionAndTasks = new ExceptionAndTask(taskCorruptedException, task1);
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
doThrow(taskCorruptedException).doReturn(0L).when(changelogReader).restore(updatingTasks);
|
||||
|
@ -697,9 +696,9 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldUpdateStandbyTaskAfterAllActiveStatefulTasksRemoved() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
@ -719,8 +718,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotSwitchTwiceToUpdatingStandbyTaskIfStandbyTaskIsRemoved() throws Exception {
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
@ -735,13 +734,13 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRemoveUpdatingActiveStatefulTask() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldRemoveUpdatingStatefulTask(task);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRemoveUpdatingStandbyTask() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldRemoveUpdatingStatefulTask(task);
|
||||
}
|
||||
|
||||
|
@ -765,7 +764,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRemovingUpdatingActiveTaskFailsWithStreamsException() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamsException streamsException = new StreamsException("Something happened", task.id());
|
||||
setupShouldThrowIfRemovingUpdatingStatefulTaskFailsWithException(task, streamsException);
|
||||
|
||||
|
@ -777,7 +776,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRemovingUpdatingActiveTaskFailsWithRuntimeException() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final RuntimeException runtimeException = new RuntimeException("Something happened");
|
||||
setupShouldThrowIfRemovingUpdatingStatefulTaskFailsWithException(task, runtimeException);
|
||||
|
||||
|
@ -788,7 +787,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRemovingUpdatingStandbyTaskFailsWithStreamsException() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StreamsException streamsException = new StreamsException("Something happened", task.id());
|
||||
setupShouldThrowIfRemovingUpdatingStatefulTaskFailsWithException(task, streamsException);
|
||||
|
||||
|
@ -799,7 +798,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRemovingUpdatingStandbyTaskFailsWithRuntimeException() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final RuntimeException runtimeException = new RuntimeException("Something happened");
|
||||
setupShouldThrowIfRemovingUpdatingStatefulTaskFailsWithException(task, runtimeException);
|
||||
|
||||
|
@ -834,8 +833,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRemovePausedTask() throws Exception {
|
||||
final StreamTask statefulTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask statefulTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(statefulTask);
|
||||
stateUpdater.add(standbyTask);
|
||||
|
@ -859,7 +858,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldThrowIfRemovingPausedTaskFails() throws Exception {
|
||||
final StreamTask statefulTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask statefulTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamsException streamsException = new StreamsException("Something happened", statefulTask.id());
|
||||
final Collection<TopicPartition> changelogPartitions = statefulTask.changelogPartitions();
|
||||
doThrow(streamsException).when(changelogReader).unregister(changelogPartitions);
|
||||
|
@ -883,7 +882,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRemoveActiveStatefulTaskFromRestoredActiveTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldRemoveTaskFromRestoredActiveTasks(task);
|
||||
}
|
||||
|
||||
|
@ -912,13 +911,13 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldRemoveActiveStatefulTaskFromFailedTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldRemoveTaskFromFailedTasks(task);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRemoveStandbyTaskFromFailedTasks() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldRemoveTaskFromFailedTasks(task);
|
||||
}
|
||||
|
||||
|
@ -947,15 +946,15 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldCompleteWithNullIfTaskNotFound() throws Exception {
|
||||
final StreamTask updatingTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask restoredTask = statefulTask(TASK_0_1, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask failedTask = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(mkSet(TASK_0_2));
|
||||
final StreamTask updatingTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask restoredTask = statefulTask(TASK_0_1, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask failedTask = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(Set.of(TASK_0_2));
|
||||
doThrow(taskCorruptedException).when(changelogReader).restore(mkMap(
|
||||
mkEntry(TASK_0_0, updatingTask),
|
||||
mkEntry(TASK_0_2, failedTask)
|
||||
));
|
||||
when(changelogReader.completedChangelogs()).thenReturn(mkSet(TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Set.of(TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(updatingTask);
|
||||
|
@ -987,22 +986,22 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldPauseActiveStatefulTask() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldPauseStatefulTask(task);
|
||||
verify(changelogReader, never()).transitToUpdateStandby();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldPauseStandbyTask() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldPauseStatefulTask(task);
|
||||
verify(changelogReader, times(1)).transitToUpdateStandby();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldPauseActiveTaskAndTransitToUpdateStandby() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_A_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_B_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_A_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_B_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
|
||||
when(topologyMetadata.isPaused(task1.id().topologyName())).thenReturn(false).thenReturn(true);
|
||||
|
||||
|
@ -1021,8 +1020,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldPauseStandbyTaskAndNotTransitToRestoreActive() throws Exception {
|
||||
final StandbyTask task1 = standbyTask(TASK_A_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_B_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task1 = standbyTask(TASK_A_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_B_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
|
||||
when(topologyMetadata.isPaused(task1.id().topologyName())).thenReturn(false).thenReturn(true);
|
||||
|
||||
|
@ -1063,8 +1062,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotPauseActiveStatefulTaskInRestoredActiveTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.singleton(TOPIC_PARTITION_A_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
|
@ -1084,18 +1083,18 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotPauseActiveStatefulTaskInFailedTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldNotPauseTaskInFailedTasks(task);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotPauseStandbyTaskInFailedTasks() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldNotPauseTaskInFailedTasks(task);
|
||||
}
|
||||
|
||||
private void shouldNotPauseTaskInFailedTasks(final Task task) throws Exception {
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
|
||||
final StreamsException streamsException = new StreamsException("Something happened", task.id());
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
|
@ -1125,14 +1124,14 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldResumeActiveStatefulTask() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldResumeStatefulTask(task);
|
||||
verify(changelogReader, times(2)).enforceRestoreActive();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldIdleWhenAllTasksPaused() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(task);
|
||||
|
||||
|
@ -1150,7 +1149,7 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldResumeStandbyTask() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldResumeStatefulTask(task);
|
||||
verify(changelogReader, times(2)).transitToUpdateStandby();
|
||||
}
|
||||
|
@ -1184,8 +1183,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotResumeActiveStatefulTaskInRestoredActiveTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.singleton(TOPIC_PARTITION_A_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
|
@ -1202,18 +1201,18 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotResumeActiveStatefulTaskInFailedTasks() throws Exception {
|
||||
final StreamTask task = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
shouldNotPauseTaskInFailedTasks(task);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotResumeStandbyTaskInFailedTasks() throws Exception {
|
||||
final StandbyTask task = standbyTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task = standbyTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
shouldNotResumeTaskInFailedTasks(task);
|
||||
}
|
||||
|
||||
private void shouldNotResumeTaskInFailedTasks(final Task task) throws Exception {
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask controlTask = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamsException streamsException = new StreamsException("Something happened", task.id());
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
|
@ -1238,8 +1237,8 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldAddFailedTasksToQueueWhenRestoreThrowsStreamsExceptionWithoutTask() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final String exceptionMessage = "The Streams were crossed!";
|
||||
final StreamsException streamsException = new StreamsException(exceptionMessage);
|
||||
final Map<TaskId, Task> updatingTasks = mkMap(
|
||||
|
@ -1263,9 +1262,9 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldAddFailedTasksToQueueWhenRestoreThrowsStreamsExceptionWithTask() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final String exceptionMessage = "The Streams were crossed!";
|
||||
final StreamsException streamsException1 = new StreamsException(exceptionMessage, task1.id());
|
||||
final StreamsException streamsException2 = new StreamsException(exceptionMessage, task3.id());
|
||||
|
@ -1301,10 +1300,10 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldHandleTaskCorruptedExceptionAndAddFailedTasksToQueue() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final Set<TaskId> expectedTaskIds = mkSet(task1.id(), task2.id());
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final Set<TaskId> expectedTaskIds = Set.of(task1.id(), task2.id());
|
||||
final TaskCorruptedException taskCorruptedException = new TaskCorruptedException(expectedTaskIds);
|
||||
final Map<TaskId, Task> updatingTasks = mkMap(
|
||||
mkEntry(task1.id(), task1),
|
||||
|
@ -1323,16 +1322,16 @@ class DefaultStateUpdaterTest {
|
|||
verifyExceptionsAndFailedTasks(expectedExceptionAndTask1, expectedExceptionAndTask2);
|
||||
verifyUpdatingTasks(task3);
|
||||
verifyRestoredActiveTasks();
|
||||
verify(changelogReader).unregister(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
verify(task1).markChangelogAsCorrupted(mkSet(TOPIC_PARTITION_A_0));
|
||||
verify(task2).markChangelogAsCorrupted(mkSet(TOPIC_PARTITION_B_0));
|
||||
verify(changelogReader).unregister(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
verify(task1).markChangelogAsCorrupted(Set.of(TOPIC_PARTITION_A_0));
|
||||
verify(task2).markChangelogAsCorrupted(Set.of(TOPIC_PARTITION_B_0));
|
||||
assertTrue(stateUpdater.isRunning());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldAddFailedTasksToQueueWhenUncaughtExceptionIsThrown() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task2 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RUNNING).build();
|
||||
final IllegalStateException illegalStateException = new IllegalStateException("Nobody expects the Spanish inquisition!");
|
||||
final Map<TaskId, Task> updatingTasks = mkMap(
|
||||
mkEntry(task1.id(), task1),
|
||||
|
@ -1358,10 +1357,10 @@ class DefaultStateUpdaterTest {
|
|||
assertFalse(stateUpdater.hasExceptionsAndFailedTasks());
|
||||
assertTrue(stateUpdater.drainExceptionsAndFailedTasks().isEmpty());
|
||||
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_1_1, mkSet(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task4 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_D_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_1_1, Set.of(TOPIC_PARTITION_C_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task3 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task4 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_D_0)).inState(State.RESTORING).build();
|
||||
final String exceptionMessage = "The Streams were crossed!";
|
||||
final StreamsException streamsException1 = new StreamsException(exceptionMessage, task1.id());
|
||||
final Map<TaskId, Task> updatingTasks1 = mkMap(
|
||||
|
@ -1406,10 +1405,10 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
@Test
|
||||
public void shouldAutoCheckpointTasksOnInterval() throws Exception {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
@ -1432,10 +1431,10 @@ class DefaultStateUpdaterTest {
|
|||
final Time time = new MockTime();
|
||||
final DefaultStateUpdater stateUpdater = new DefaultStateUpdater("test-state-updater", metrics, config, null, changelogReader, topologyMetadata, time);
|
||||
try {
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StreamTask task1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask task2 = statefulTask(TASK_0_2, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask task3 = standbyTask(TASK_1_0, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask task4 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
@ -1468,11 +1467,11 @@ class DefaultStateUpdaterTest {
|
|||
public void shouldTasksFromInputQueue() {
|
||||
stateUpdater.shutdown(Duration.ofMillis(Long.MAX_VALUE));
|
||||
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
stateUpdater.add(activeTask1);
|
||||
stateUpdater.add(standbyTask1);
|
||||
stateUpdater.add(standbyTask2);
|
||||
|
@ -1480,16 +1479,16 @@ class DefaultStateUpdaterTest {
|
|||
stateUpdater.add(activeTask2);
|
||||
stateUpdater.add(standbyTask3);
|
||||
|
||||
verifyGetTasks(mkSet(activeTask1, activeTask2), mkSet(standbyTask1, standbyTask2, standbyTask3));
|
||||
verifyGetTasks(Set.of(activeTask1, activeTask2), Set.of(standbyTask1, standbyTask2, standbyTask3));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldTasks() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_2, mkSet(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_2, Set.of(TOPIC_PARTITION_C_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Collections.emptySet());
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
|
@ -1500,34 +1499,34 @@ class DefaultStateUpdaterTest {
|
|||
stateUpdater.add(standbyTask3);
|
||||
verifyUpdatingTasks(activeTask1, activeTask2, standbyTask1, standbyTask2, standbyTask3);
|
||||
|
||||
verifyGetTasks(mkSet(activeTask1, activeTask2), mkSet(standbyTask1, standbyTask2, standbyTask3));
|
||||
verifyGetTasks(Set.of(activeTask1, activeTask2), Set.of(standbyTask1, standbyTask2, standbyTask3));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldGetTasksFromRestoredActiveTasks() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(mkSet(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
final StreamTask activeTask1 = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
when(changelogReader.completedChangelogs()).thenReturn(Set.of(TOPIC_PARTITION_A_0, TOPIC_PARTITION_B_0));
|
||||
when(changelogReader.allChangelogsCompleted()).thenReturn(false);
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(activeTask1);
|
||||
stateUpdater.add(activeTask2);
|
||||
verifyRestoredActiveTasks(activeTask1, activeTask2);
|
||||
|
||||
verifyGetTasks(mkSet(activeTask1, activeTask2), mkSet());
|
||||
verifyGetTasks(Set.of(activeTask1, activeTask2), Set.of());
|
||||
|
||||
stateUpdater.drainRestoredActiveTasks(Duration.ofMinutes(1));
|
||||
|
||||
verifyGetTasks(mkSet(), mkSet());
|
||||
verifyGetTasks(Set.of(), Set.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldGetTasksFromExceptionsAndFailedTasks() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_1_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, mkSet(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_1_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask2 = standbyTask(TASK_1_1, Set.of(TOPIC_PARTITION_D_0)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask1 = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final TaskCorruptedException taskCorruptedException =
|
||||
new TaskCorruptedException(mkSet(standbyTask1.id(), standbyTask2.id()));
|
||||
new TaskCorruptedException(Set.of(standbyTask1.id(), standbyTask2.id()));
|
||||
final StreamsException streamsException = new StreamsException("The Streams were crossed!", activeTask1.id());
|
||||
final Map<TaskId, Task> updatingTasks1 = mkMap(
|
||||
mkEntry(activeTask1.id(), activeTask1),
|
||||
|
@ -1548,17 +1547,17 @@ class DefaultStateUpdaterTest {
|
|||
final ExceptionAndTask expectedExceptionAndTasks3 = new ExceptionAndTask(streamsException, activeTask1);
|
||||
verifyExceptionsAndFailedTasks(expectedExceptionAndTasks1, expectedExceptionAndTasks2, expectedExceptionAndTasks3);
|
||||
|
||||
verifyGetTasks(mkSet(activeTask1), mkSet(standbyTask1, standbyTask2));
|
||||
verifyGetTasks(Set.of(activeTask1), Set.of(standbyTask1, standbyTask2));
|
||||
|
||||
stateUpdater.drainExceptionsAndFailedTasks();
|
||||
|
||||
verifyGetTasks(mkSet(), mkSet());
|
||||
verifyGetTasks(Set.of(), Set.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldGetTasksFromPausedTasks() throws Exception {
|
||||
final StreamTask activeTask = statefulTask(TASK_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_1, mkSet(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask = statefulTask(TASK_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask = standbyTask(TASK_0_1, Set.of(TOPIC_PARTITION_A_0)).inState(State.RUNNING).build();
|
||||
stateUpdater.start();
|
||||
stateUpdater.add(activeTask);
|
||||
stateUpdater.add(standbyTask);
|
||||
|
@ -1568,15 +1567,15 @@ class DefaultStateUpdaterTest {
|
|||
|
||||
verifyPausedTasks(activeTask, standbyTask);
|
||||
|
||||
verifyGetTasks(mkSet(activeTask), mkSet(standbyTask));
|
||||
verifyGetTasks(Set.of(activeTask), Set.of(standbyTask));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldRecordMetrics() throws Exception {
|
||||
final StreamTask activeTask1 = statefulTask(TASK_A_0_0, mkSet(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_B_0_0, mkSet(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_A_0_1, mkSet(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask4 = standbyTask(TASK_B_0_1, mkSet(TOPIC_PARTITION_B_1)).inState(State.RUNNING).build();
|
||||
final StreamTask activeTask1 = statefulTask(TASK_A_0_0, Set.of(TOPIC_PARTITION_A_0)).inState(State.RESTORING).build();
|
||||
final StreamTask activeTask2 = statefulTask(TASK_B_0_0, Set.of(TOPIC_PARTITION_B_0)).inState(State.RESTORING).build();
|
||||
final StandbyTask standbyTask3 = standbyTask(TASK_A_0_1, Set.of(TOPIC_PARTITION_A_1)).inState(State.RUNNING).build();
|
||||
final StandbyTask standbyTask4 = standbyTask(TASK_B_0_1, Set.of(TOPIC_PARTITION_B_1)).inState(State.RUNNING).build();
|
||||
final Map<TaskId, Task> tasks1234 = mkMap(
|
||||
mkEntry(activeTask1.id(), activeTask1),
|
||||
mkEntry(activeTask2.id(), activeTask2),
|
||||
|
@ -1710,7 +1709,7 @@ class DefaultStateUpdaterTest {
|
|||
"Did not get empty restored active task within the given timeout!"
|
||||
);
|
||||
} else {
|
||||
final Set<StreamTask> expectedRestoredTasks = mkSet(tasks);
|
||||
final Set<StreamTask> expectedRestoredTasks = Set.of(tasks);
|
||||
final Set<StreamTask> restoredTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
@ -1725,7 +1724,7 @@ class DefaultStateUpdaterTest {
|
|||
}
|
||||
|
||||
private void verifyDrainingRestoredActiveTasks(final StreamTask... tasks) throws Exception {
|
||||
final Set<StreamTask> expectedRestoredTasks = mkSet(tasks);
|
||||
final Set<StreamTask> expectedRestoredTasks = Set.of(tasks);
|
||||
final Set<StreamTask> restoredTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
@ -1747,7 +1746,7 @@ class DefaultStateUpdaterTest {
|
|||
"Did not get empty updating task within the given timeout!"
|
||||
);
|
||||
} else {
|
||||
final Set<Task> expectedUpdatingTasks = mkSet(tasks);
|
||||
final Set<Task> expectedUpdatingTasks = Set.of(tasks);
|
||||
final Set<Task> updatingTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
@ -1762,7 +1761,7 @@ class DefaultStateUpdaterTest {
|
|||
}
|
||||
|
||||
private void verifyUpdatingStandbyTasks(final StandbyTask... tasks) throws Exception {
|
||||
final Set<StandbyTask> expectedStandbyTasks = mkSet(tasks);
|
||||
final Set<StandbyTask> expectedStandbyTasks = Set.of(tasks);
|
||||
final Set<StandbyTask> standbyTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
@ -1791,7 +1790,7 @@ class DefaultStateUpdaterTest {
|
|||
"Did not get empty paused task within the given timeout!"
|
||||
);
|
||||
} else {
|
||||
final Set<Task> expectedPausedTasks = mkSet(tasks);
|
||||
final Set<Task> expectedPausedTasks = Set.of(tasks);
|
||||
final Set<Task> pausedTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
@ -1865,7 +1864,7 @@ class DefaultStateUpdaterTest {
|
|||
"Did not get empty removed task within the given timeout!"
|
||||
);
|
||||
} else {
|
||||
final Set<Task> expectedRemovedTasks = mkSet(tasks);
|
||||
final Set<Task> expectedRemovedTasks = Set.of(tasks);
|
||||
final Set<Task> removedTasks = new HashSet<>();
|
||||
waitForCondition(
|
||||
() -> {
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.apache.kafka.common.serialization.Deserializer;
|
|||
import org.apache.kafka.common.utils.LogCaptureAppender;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.KeyValue;
|
||||
import org.apache.kafka.streams.StreamsConfig;
|
||||
import org.apache.kafka.streams.errors.DeserializationExceptionHandler;
|
||||
|
@ -251,7 +250,7 @@ public class GlobalStateManagerImplTest {
|
|||
@Test
|
||||
public void shouldReturnInitializedStoreNames() {
|
||||
final Set<String> storeNames = stateManager.initialize();
|
||||
assertEquals(Utils.mkSet(storeName1, storeName2, storeName3, storeName4, storeName5), storeNames);
|
||||
assertEquals(Set.of(storeName1, storeName2, storeName3, storeName4, storeName5), storeNames);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -26,7 +26,6 @@ import org.apache.kafka.common.serialization.LongSerializer;
|
|||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
import org.apache.kafka.common.utils.LogContext;
|
||||
import org.apache.kafka.common.utils.MockTime;
|
||||
import org.apache.kafka.common.utils.Utils;
|
||||
import org.apache.kafka.streams.errors.LogAndContinueExceptionHandler;
|
||||
import org.apache.kafka.streams.errors.LogAndFailExceptionHandler;
|
||||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
|
@ -84,7 +83,7 @@ public class GlobalStateTaskTest {
|
|||
|
||||
@BeforeEach
|
||||
public void before() {
|
||||
final Set<String> storeNames = Utils.mkSet("t1-store", "t2-store");
|
||||
final Set<String> storeNames = Set.of("t1-store", "t2-store");
|
||||
final Map<String, SourceNode<?, ?>> sourceByTopics = new HashMap<>();
|
||||
sourceByTopics.put(topic1, sourceOne);
|
||||
sourceByTopics.put(topic2, sourceTwo);
|
||||
|
|
|
@ -59,7 +59,6 @@ import static java.util.Arrays.asList;
|
|||
import static java.util.Collections.emptySet;
|
||||
import static java.util.Collections.singletonList;
|
||||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.EMPTY_CHANGELOG_END_OFFSETS;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.EMPTY_TASKS;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.PID_1;
|
||||
|
@ -172,7 +171,7 @@ public class HighAvailabilityStreamsPartitionAssignorTest {
|
|||
builder.addSource(null, "source1", null, null, null, "topic1");
|
||||
builder.addProcessor("processor1", new MockApiProcessorSupplier<>(), "source1");
|
||||
builder.addStateStore(new MockKeyValueStoreBuilder("store1", false), "processor1");
|
||||
final Set<TaskId> allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> allTasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
|
||||
createMockTaskManager();
|
||||
when(adminClient.listOffsets(any())).thenThrow(new StreamsException("Should be handled"));
|
||||
|
@ -224,7 +223,7 @@ public class HighAvailabilityStreamsPartitionAssignorTest {
|
|||
builder.addSource(null, "source1", null, null, null, "topic1");
|
||||
builder.addProcessor("processor1", new MockApiProcessorSupplier<>(), "source1");
|
||||
builder.addStateStore(new MockKeyValueStoreBuilder("store1", false), "processor1");
|
||||
final Set<TaskId> allTasks = mkSet(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
final Set<TaskId> allTasks = Set.of(TASK_0_0, TASK_0_1, TASK_0_2);
|
||||
|
||||
createMockTaskManager();
|
||||
adminClient = createMockAdminClientForAssignor(getTopicPartitionOffsetsMap(
|
||||
|
|
|
@ -76,7 +76,6 @@ import java.util.stream.Collectors;
|
|||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.anEmptyMap;
|
||||
import static org.hamcrest.Matchers.empty;
|
||||
|
@ -185,12 +184,12 @@ public class InternalTopicManagerTest {
|
|||
final InternalTopicConfig internalTopicConfig2 = setupRepartitionTopicConfig(topic2, 1);
|
||||
final NewTopic newTopic1 = newTopic(topic1, internalTopicConfig1, streamsConfig);
|
||||
final NewTopic newTopic2 = newTopic(topic2, internalTopicConfig2, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic1, newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic1, newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicSuccessfulFuture),
|
||||
mkEntry(topic2, createTopicFailFuture)
|
||||
)));
|
||||
when(admin.createTopics(mkSet(newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic2, createTopicSuccessfulFuture)
|
||||
)));
|
||||
|
@ -223,7 +222,7 @@ public class InternalTopicManagerTest {
|
|||
);
|
||||
final InternalTopicConfig internalTopicConfig = setupRepartitionTopicConfig(topic1, 1);
|
||||
final NewTopic newTopic = newTopic(topic1, internalTopicConfig, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic)))
|
||||
when(admin.createTopics(Set.of(newTopic)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicSuccessfulFuture)
|
||||
)))
|
||||
|
@ -394,7 +393,7 @@ public class InternalTopicManagerTest {
|
|||
final KafkaFutureImpl<TopicMetadataAndConfig> createTopicFailFuture = new KafkaFutureImpl<>();
|
||||
createTopicFailFuture.completeExceptionally(new IllegalStateException("Nobody expects the Spanish inquisition"));
|
||||
final NewTopic newTopic = newTopic(topic1, internalTopicConfig, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic)))
|
||||
when(admin.createTopics(Set.of(newTopic)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicFailFuture)
|
||||
)));
|
||||
|
@ -411,7 +410,7 @@ public class InternalTopicManagerTest {
|
|||
final InternalTopicManager topicManager = new InternalTopicManager(time, admin, streamsConfig);
|
||||
final InternalTopicConfig internalTopicConfig = setupRepartitionTopicConfig(topic1, 1);
|
||||
final NewTopic newTopic = newTopic(topic1, internalTopicConfig, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic)))
|
||||
when(admin.createTopics(Set.of(newTopic)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(Collections.singletonMap(topic2, new KafkaFutureImpl<>())));
|
||||
|
||||
assertThrows(
|
||||
|
@ -432,7 +431,7 @@ public class InternalTopicManagerTest {
|
|||
createTopicFailFuture.completeExceptionally(new TimeoutException());
|
||||
final InternalTopicConfig internalTopicConfig = setupRepartitionTopicConfig(topic1, 1);
|
||||
final NewTopic newTopic = newTopic(topic1, internalTopicConfig, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic)))
|
||||
when(admin.createTopics(Set.of(newTopic)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(mkEntry(topic1, createTopicFailFuture))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -452,7 +451,7 @@ public class InternalTopicManagerTest {
|
|||
final KafkaFutureImpl<TopicMetadataAndConfig> createTopicFutureThatNeverCompletes = new KafkaFutureImpl<>();
|
||||
final InternalTopicConfig internalTopicConfig = setupRepartitionTopicConfig(topic1, 1);
|
||||
final NewTopic newTopic = newTopic(topic1, internalTopicConfig, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic)))
|
||||
when(admin.createTopics(Set.of(newTopic)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(mkEntry(topic1, createTopicFutureThatNeverCompletes))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -474,7 +473,7 @@ public class InternalTopicManagerTest {
|
|||
setupCleanUpScenario(admin, streamsConfig, internalTopicConfig1, internalTopicConfig2);
|
||||
final KafkaFutureImpl<Void> deleteTopicSuccessfulFuture = new KafkaFutureImpl<>();
|
||||
deleteTopicSuccessfulFuture.complete(null);
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicSuccessfulFuture))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -501,18 +500,18 @@ public class InternalTopicManagerTest {
|
|||
);
|
||||
final NewTopic newTopic1 = newTopic(topic1, internalTopicConfig1, streamsConfig);
|
||||
final NewTopic newTopic2 = newTopic(topic2, internalTopicConfig2, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic1, newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic1, newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicSuccessfulFuture),
|
||||
mkEntry(topic2, createTopicFailFuture1)
|
||||
)));
|
||||
when(admin.createTopics(mkSet(newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic3, createTopicSuccessfulFuture)
|
||||
)));
|
||||
final KafkaFutureImpl<Void> deleteTopicSuccessfulFuture = new KafkaFutureImpl<>();
|
||||
deleteTopicSuccessfulFuture.complete(null);
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicSuccessfulFuture))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -542,17 +541,17 @@ public class InternalTopicManagerTest {
|
|||
);
|
||||
final NewTopic newTopic1 = newTopic(topic1, internalTopicConfig1, streamsConfig);
|
||||
final NewTopic newTopic2 = newTopic(topic2, internalTopicConfig2, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic1, newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic1, newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicSuccessfulFuture),
|
||||
mkEntry(topic2, createTopicFailFuture1)
|
||||
)));
|
||||
final KafkaFutureImpl<TopicMetadataAndConfig> createTopicFutureThatNeverCompletes = new KafkaFutureImpl<>();
|
||||
when(admin.createTopics(mkSet(newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(mkEntry(topic2, createTopicFutureThatNeverCompletes))));
|
||||
final KafkaFutureImpl<Void> deleteTopicSuccessfulFuture = new KafkaFutureImpl<>();
|
||||
deleteTopicSuccessfulFuture.complete(null);
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicSuccessfulFuture))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -590,7 +589,7 @@ public class InternalTopicManagerTest {
|
|||
deleteTopicFailFuture.completeExceptionally(retriableException);
|
||||
final KafkaFutureImpl<Void> deleteTopicSuccessfulFuture = new KafkaFutureImpl<>();
|
||||
deleteTopicSuccessfulFuture.complete(null);
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicFailFuture))))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicSuccessfulFuture))));
|
||||
|
||||
|
@ -615,7 +614,7 @@ public class InternalTopicManagerTest {
|
|||
final InternalTopicConfig internalTopicConfig2 = setupRepartitionTopicConfig(topic2, 1);
|
||||
setupCleanUpScenario(admin, streamsConfig, internalTopicConfig1, internalTopicConfig2);
|
||||
final KafkaFutureImpl<Void> deleteTopicFutureThatNeverCompletes = new KafkaFutureImpl<>();
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicFutureThatNeverCompletes))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -637,7 +636,7 @@ public class InternalTopicManagerTest {
|
|||
setupCleanUpScenario(admin, streamsConfig, internalTopicConfig1, internalTopicConfig2);
|
||||
final KafkaFutureImpl<Void> deleteTopicFailFuture = new KafkaFutureImpl<>();
|
||||
deleteTopicFailFuture.completeExceptionally(new IllegalStateException("Nobody expects the Spanish inquisition"));
|
||||
when(admin.deleteTopics(mkSet(topic1)))
|
||||
when(admin.deleteTopics(Set.of(topic1)))
|
||||
.thenAnswer(answer -> new MockDeleteTopicsResult(mkMap(mkEntry(topic1, deleteTopicFailFuture))));
|
||||
|
||||
assertThrows(
|
||||
|
@ -660,12 +659,12 @@ public class InternalTopicManagerTest {
|
|||
);
|
||||
final NewTopic newTopic1 = newTopic(topic1, internalTopicConfig1, streamsConfig);
|
||||
final NewTopic newTopic2 = newTopic(topic2, internalTopicConfig2, streamsConfig);
|
||||
when(admin.createTopics(mkSet(newTopic1, newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic1, newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic1, createTopicSuccessfulFuture),
|
||||
mkEntry(topic2, createTopicFailFuture1)
|
||||
)));
|
||||
when(admin.createTopics(mkSet(newTopic2)))
|
||||
when(admin.createTopics(Set.of(newTopic2)))
|
||||
.thenAnswer(answer -> new MockCreateTopicsResult(mkMap(
|
||||
mkEntry(topic2, createTopicFailFuture2)
|
||||
)));
|
||||
|
@ -711,7 +710,7 @@ public class InternalTopicManagerTest {
|
|||
internalTopicManager.makeReady(Collections.singletonMap(topic3, topicConfig3));
|
||||
internalTopicManager.makeReady(Collections.singletonMap(topic4, topicConfig4));
|
||||
|
||||
assertEquals(mkSet(topic1, topic2, topic3, topic4), mockAdminClient.listTopics().names().get());
|
||||
assertEquals(Set.of(topic1, topic2, topic3, topic4), mockAdminClient.listTopics().names().get());
|
||||
assertEquals(new TopicDescription(topic1, false, new ArrayList<TopicPartitionInfo>() {
|
||||
{
|
||||
add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()));
|
||||
|
@ -779,7 +778,7 @@ public class InternalTopicManagerTest {
|
|||
|
||||
// let the first describe succeed on topic, and fail on topic2, and then let creation throws topics-existed;
|
||||
// it should retry with just topic2 and then let it succeed
|
||||
when(admin.describeTopics(mkSet(topic1, topic2)))
|
||||
when(admin.describeTopics(Set.of(topic1, topic2)))
|
||||
.thenAnswer(answer -> new MockDescribeTopicsResult(mkMap(
|
||||
mkEntry(topic1, topicDescriptionSuccessFuture),
|
||||
mkEntry(topic2, topicDescriptionFailFuture)
|
||||
|
@ -1491,12 +1490,12 @@ public class InternalTopicManagerTest {
|
|||
false,
|
||||
Collections.singletonList(new TopicPartitionInfo(0, broker1, cluster, Collections.emptyList()))
|
||||
));
|
||||
when(admin.describeTopics(mkSet(topic1, topic2)))
|
||||
when(admin.describeTopics(Set.of(topic1, topic2)))
|
||||
.thenAnswer(answer -> new MockDescribeTopicsResult(mkMap(
|
||||
mkEntry(topic1, topicDescriptionSuccessfulFuture1),
|
||||
mkEntry(topic2, topicDescriptionFailFuture)
|
||||
)));
|
||||
when(admin.describeTopics(mkSet(topic2)))
|
||||
when(admin.describeTopics(Set.of(topic2)))
|
||||
.thenAnswer(answer -> new MockDescribeTopicsResult(mkMap(
|
||||
mkEntry(topic2, topicDescriptionSuccessfulFuture2)
|
||||
)));
|
||||
|
@ -1507,7 +1506,7 @@ public class InternalTopicManagerTest {
|
|||
);
|
||||
final ConfigResource topicResource1 = new ConfigResource(Type.TOPIC, topic1);
|
||||
final ConfigResource topicResource2 = new ConfigResource(Type.TOPIC, topic2);
|
||||
when(admin.describeConfigs(mkSet(topicResource1, topicResource2)))
|
||||
when(admin.describeConfigs(Set.of(topicResource1, topicResource2)))
|
||||
.thenAnswer(answer -> new MockDescribeConfigsResult(mkMap(
|
||||
mkEntry(topicResource1, topicConfigSuccessfulFuture),
|
||||
mkEntry(topicResource2, topicConfigSuccessfulFuture)
|
||||
|
|
|
@ -63,7 +63,6 @@ import static java.util.Arrays.asList;
|
|||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkProperties;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_2;
|
||||
|
@ -584,10 +583,10 @@ public class InternalTopologyBuilderTest {
|
|||
|
||||
builder.buildTopology();
|
||||
final Set<String> stateStoreNames = builder.stateStoreNamesForSubtopology(0);
|
||||
assertThat(stateStoreNames, equalTo(mkSet(storeBuilder.name())));
|
||||
assertThat(stateStoreNames, equalTo(Set.of(storeBuilder.name())));
|
||||
|
||||
final Set<String> emptyStoreNames = builder.stateStoreNamesForSubtopology(1);
|
||||
assertThat(emptyStoreNames, equalTo(mkSet()));
|
||||
assertThat(emptyStoreNames, equalTo(Set.of()));
|
||||
|
||||
final Set<String> stateStoreNamesUnknownSubtopology = builder.stateStoreNamesForSubtopology(13);
|
||||
assertThat(stateStoreNamesUnknownSubtopology, nullValue());
|
||||
|
@ -629,16 +628,16 @@ public class InternalTopologyBuilderTest {
|
|||
final Map<Subtopology, InternalTopologyBuilder.TopicsInfo> topicGroups = builder.subtopologyToTopicsInfo();
|
||||
|
||||
final Map<Subtopology, InternalTopologyBuilder.TopicsInfo> expectedTopicGroups = new HashMap<>();
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_0, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-1", "X-topic-1x", "topic-2"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_1, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_2, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_0, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), Set.of("topic-1", "X-topic-1x", "topic-2"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_1, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), Set.of("topic-3", "topic-4"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_2, new InternalTopologyBuilder.TopicsInfo(Collections.emptySet(), Set.of("topic-5"), Collections.emptyMap(), Collections.emptyMap()));
|
||||
|
||||
assertEquals(3, topicGroups.size());
|
||||
assertEquals(expectedTopicGroups, topicGroups);
|
||||
|
||||
final Collection<Set<String>> copartitionGroups = builder.copartitionGroups();
|
||||
|
||||
assertEquals(mkSet(mkSet("topic-1", "X-topic-1x", "topic-2")), new HashSet<>(copartitionGroups));
|
||||
assertEquals(Set.of(Set.of("topic-1", "X-topic-1x", "topic-2")), new HashSet<>(copartitionGroups));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -670,15 +669,15 @@ public class InternalTopologyBuilderTest {
|
|||
final String store2 = ProcessorStateManager.storeChangelogTopic("X", "store-2", builder.topologyName());
|
||||
final String store3 = ProcessorStateManager.storeChangelogTopic("X", "store-3", builder.topologyName());
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_0, new InternalTopologyBuilder.TopicsInfo(
|
||||
Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"),
|
||||
Collections.emptySet(), Set.of("topic-1", "topic-1x", "topic-2"),
|
||||
Collections.emptyMap(),
|
||||
Collections.singletonMap(store1, new UnwindowedUnversionedChangelogTopicConfig(store1, Collections.emptyMap()))));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_1, new InternalTopologyBuilder.TopicsInfo(
|
||||
Collections.emptySet(), mkSet("topic-3", "topic-4"),
|
||||
Collections.emptySet(), Set.of("topic-3", "topic-4"),
|
||||
Collections.emptyMap(),
|
||||
Collections.singletonMap(store2, new UnwindowedUnversionedChangelogTopicConfig(store2, Collections.emptyMap()))));
|
||||
expectedTopicGroups.put(SUBTOPOLOGY_2, new InternalTopologyBuilder.TopicsInfo(
|
||||
Collections.emptySet(), mkSet("topic-5"),
|
||||
Collections.emptySet(), Set.of("topic-5"),
|
||||
Collections.emptyMap(),
|
||||
Collections.singletonMap(store3, new UnwindowedUnversionedChangelogTopicConfig(store3, Collections.emptyMap()))));
|
||||
|
||||
|
@ -703,9 +702,9 @@ public class InternalTopologyBuilderTest {
|
|||
final ProcessorTopology topology1 = builder.buildSubtopology(1);
|
||||
final ProcessorTopology topology2 = builder.buildSubtopology(2);
|
||||
|
||||
assertEquals(mkSet("source-1", "source-2", "processor-1", "processor-2"), nodeNames(topology0.processors()));
|
||||
assertEquals(mkSet("source-3", "source-4", "processor-3"), nodeNames(topology1.processors()));
|
||||
assertEquals(mkSet("source-5"), nodeNames(topology2.processors()));
|
||||
assertEquals(Set.of("source-1", "source-2", "processor-1", "processor-2"), nodeNames(topology0.processors()));
|
||||
assertEquals(Set.of("source-3", "source-4", "processor-3"), nodeNames(topology1.processors()));
|
||||
assertEquals(Set.of("source-5"), nodeNames(topology2.processors()));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -46,11 +46,11 @@ import java.util.Arrays;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
@ -481,7 +481,7 @@ public class PartitionGroupTest {
|
|||
group.nextRecord(new RecordInfo(), time.milliseconds());
|
||||
|
||||
// shrink list of queues
|
||||
group.updatePartitions(mkSet(createPartition2()), p -> {
|
||||
group.updatePartitions(Set.of(createPartition2()), p -> {
|
||||
fail("should not create any queues");
|
||||
return null;
|
||||
});
|
||||
|
@ -514,7 +514,7 @@ public class PartitionGroupTest {
|
|||
group.nextRecord(new RecordInfo(), time.milliseconds());
|
||||
|
||||
// expand list of queues
|
||||
group.updatePartitions(mkSet(createPartition1(), createPartition2()), p -> {
|
||||
group.updatePartitions(Set.of(createPartition1(), createPartition2()), p -> {
|
||||
assertEquals(createPartition2(), p);
|
||||
return createQueue2();
|
||||
});
|
||||
|
@ -546,7 +546,7 @@ public class PartitionGroupTest {
|
|||
group.nextRecord(new RecordInfo(), time.milliseconds());
|
||||
|
||||
// expand and shrink list of queues
|
||||
group.updatePartitions(mkSet(createPartition2()), p -> {
|
||||
group.updatePartitions(Set.of(createPartition2()), p -> {
|
||||
assertEquals(createPartition2(), p);
|
||||
return createQueue2();
|
||||
});
|
||||
|
|
|
@ -32,7 +32,6 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
@ -61,14 +60,14 @@ public class PartitionGrouperTest {
|
|||
final Map<TaskId, Set<TopicPartition>> expectedPartitionsForTask = new HashMap<>();
|
||||
final Map<Subtopology, Set<String>> topicGroups = new HashMap<>();
|
||||
|
||||
topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1"));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 0, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 0)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 1, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 1)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 2, SUBTOPOLOGY_0.namedTopology), mkSet(new TopicPartition("topic1", 2)));
|
||||
topicGroups.put(SUBTOPOLOGY_0, Set.of("topic1"));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 0, SUBTOPOLOGY_0.namedTopology), Set.of(new TopicPartition("topic1", 0)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 1, SUBTOPOLOGY_0.namedTopology), Set.of(new TopicPartition("topic1", 1)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_0.nodeGroupId, 2, SUBTOPOLOGY_0.namedTopology), Set.of(new TopicPartition("topic1", 2)));
|
||||
|
||||
topicGroups.put(SUBTOPOLOGY_1, mkSet("topic2"));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 0, SUBTOPOLOGY_1.namedTopology), mkSet(new TopicPartition("topic2", 0)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 1, SUBTOPOLOGY_1.namedTopology), mkSet(new TopicPartition("topic2", 1)));
|
||||
topicGroups.put(SUBTOPOLOGY_1, Set.of("topic2"));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 0, SUBTOPOLOGY_1.namedTopology), Set.of(new TopicPartition("topic2", 0)));
|
||||
expectedPartitionsForTask.put(new TaskId(SUBTOPOLOGY_1.nodeGroupId, 1, SUBTOPOLOGY_1.namedTopology), Set.of(new TopicPartition("topic2", 1)));
|
||||
|
||||
assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata));
|
||||
}
|
||||
|
@ -79,16 +78,16 @@ public class PartitionGrouperTest {
|
|||
final Map<TaskId, Set<TopicPartition>> expectedPartitionsForTask = new HashMap<>();
|
||||
final Map<Subtopology, Set<String>> topicGroups = new HashMap<>();
|
||||
|
||||
topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1", "topic2"));
|
||||
topicGroups.put(SUBTOPOLOGY_0, Set.of("topic1", "topic2"));
|
||||
expectedPartitionsForTask.put(
|
||||
new TaskId(SUBTOPOLOGY_0.nodeGroupId, 0, SUBTOPOLOGY_0.namedTopology),
|
||||
mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0)));
|
||||
Set.of(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0)));
|
||||
expectedPartitionsForTask.put(
|
||||
new TaskId(SUBTOPOLOGY_0.nodeGroupId, 1, SUBTOPOLOGY_0.namedTopology),
|
||||
mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1)));
|
||||
Set.of(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1)));
|
||||
expectedPartitionsForTask.put(
|
||||
new TaskId(SUBTOPOLOGY_0.nodeGroupId, 2, SUBTOPOLOGY_0.namedTopology),
|
||||
mkSet(new TopicPartition("topic1", 2)));
|
||||
Set.of(new TopicPartition("topic1", 2)));
|
||||
|
||||
assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata));
|
||||
}
|
||||
|
@ -98,7 +97,7 @@ public class PartitionGrouperTest {
|
|||
final PartitionGrouper grouper = new PartitionGrouper();
|
||||
final Map<Subtopology, Set<String>> topicGroups = new HashMap<>();
|
||||
|
||||
topicGroups.put(SUBTOPOLOGY_0, mkSet("topic1", "unknownTopic", "topic2"));
|
||||
topicGroups.put(SUBTOPOLOGY_0, Set.of("topic1", "unknownTopic", "topic2"));
|
||||
assertThrows(RuntimeException.class, () -> grouper.partitionGroups(topicGroups, metadata));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -62,6 +62,7 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.OptionalLong;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
|
@ -70,7 +71,6 @@ import static java.util.Collections.singletonList;
|
|||
import static java.util.Collections.singletonMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.StateManagerUtil.CHECKPOINT_FILE_NAME;
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
@ -206,7 +206,7 @@ public class ProcessorStateManagerTest {
|
|||
mkEntry(persistentStoreTwoName, persistentStoreTwoTopicName),
|
||||
mkEntry(nonPersistentStoreName, nonPersistentStoreTopicName)
|
||||
),
|
||||
mkSet(persistentStorePartition, nonPersistentStorePartition),
|
||||
Set.of(persistentStorePartition, nonPersistentStorePartition),
|
||||
false);
|
||||
|
||||
assertTrue(stateMgr.changelogAsSource(persistentStorePartition));
|
||||
|
@ -431,7 +431,7 @@ public class ProcessorStateManagerTest {
|
|||
stateMgr.initializeStoreOffsetsFromCheckpoint(true);
|
||||
|
||||
assertTrue(checkpointFile.exists());
|
||||
assertEquals(mkSet(
|
||||
assertEquals(Set.of(
|
||||
persistentStorePartition,
|
||||
persistentStoreTwoPartition,
|
||||
nonPersistentStorePartition),
|
||||
|
@ -472,7 +472,7 @@ public class ProcessorStateManagerTest {
|
|||
stateMgr.initializeStoreOffsetsFromCheckpoint(true);
|
||||
|
||||
assertFalse(checkpointFile.exists());
|
||||
assertEquals(mkSet(
|
||||
assertEquals(Set.of(
|
||||
persistentStorePartition,
|
||||
persistentStoreTwoPartition,
|
||||
nonPersistentStorePartition),
|
||||
|
@ -997,7 +997,7 @@ public class ProcessorStateManagerTest {
|
|||
|
||||
try {
|
||||
stateMgr.registerStore(persistentStore, persistentStore.stateRestoreCallback, null);
|
||||
stateMgr.markChangelogAsCorrupted(mkSet(persistentStorePartition));
|
||||
stateMgr.markChangelogAsCorrupted(Set.of(persistentStorePartition));
|
||||
|
||||
final ProcessorStateException thrown = assertThrows(ProcessorStateException.class, () -> stateMgr.initializeStoreOffsetsFromCheckpoint(true));
|
||||
assertInstanceOf(IllegalStateException.class, thrown.getCause());
|
||||
|
|
|
@ -68,7 +68,6 @@ import java.util.function.Supplier;
|
|||
import static java.util.Arrays.asList;
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.hamcrest.CoreMatchers.containsString;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.CoreMatchers.is;
|
||||
|
@ -172,7 +171,7 @@ public class ProcessorTopologyTest {
|
|||
|
||||
final ProcessorTopology processorTopology = topology.getInternalBuilder("X").buildTopology();
|
||||
|
||||
assertThat(processorTopology.terminalNodes(), equalTo(mkSet("processor-2", "sink-1")));
|
||||
assertThat(processorTopology.terminalNodes(), equalTo(Set.of("processor-2", "sink-1")));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -27,9 +27,9 @@ import java.lang.reflect.Method;
|
|||
import java.util.Collections;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.function.Consumer;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.standbyTask;
|
||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statefulTask;
|
||||
import static org.apache.kafka.test.StreamsTestUtils.TaskBuilder.statelessTask;
|
||||
|
@ -134,7 +134,7 @@ class ReadOnlyTaskTest {
|
|||
@Test
|
||||
public void shouldDelegateCommitNeededIfStandby() {
|
||||
final StandbyTask standbyTask =
|
||||
standbyTask(new TaskId(1, 0), mkSet(new TopicPartition("topic", 0))).build();
|
||||
standbyTask(new TaskId(1, 0), Set.of(new TopicPartition("topic", 0))).build();
|
||||
final ReadOnlyTask readOnlyTask = new ReadOnlyTask(standbyTask);
|
||||
|
||||
readOnlyTask.commitNeeded();
|
||||
|
@ -145,7 +145,7 @@ class ReadOnlyTaskTest {
|
|||
@Test
|
||||
public void shouldThrowUnsupportedOperationExceptionForCommitNeededIfActive() {
|
||||
final StreamTask statefulTask =
|
||||
statefulTask(new TaskId(1, 0), mkSet(new TopicPartition("topic", 0))).build();
|
||||
statefulTask(new TaskId(1, 0), Set.of(new TopicPartition("topic", 0))).build();
|
||||
final ReadOnlyTask readOnlyTask = new ReadOnlyTask(statefulTask);
|
||||
|
||||
final Exception exception = assertThrows(UnsupportedOperationException.class, readOnlyTask::commitNeeded);
|
||||
|
|
|
@ -38,13 +38,13 @@ import org.mockito.quality.Strictness;
|
|||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import static org.apache.kafka.common.utils.Utils.mkEntry;
|
||||
import static org.apache.kafka.common.utils.Utils.mkMap;
|
||||
import static org.apache.kafka.common.utils.Utils.mkSet;
|
||||
import static org.apache.kafka.streams.processor.internals.TopologyMetadata.UNNAMED_TOPOLOGY;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_0;
|
||||
import static org.apache.kafka.streams.processor.internals.assignment.AssignmentTestUtils.SUBTOPOLOGY_1;
|
||||
|
@ -82,8 +82,8 @@ public class RepartitionTopicsTest {
|
|||
private static final RepartitionTopicConfig REPARTITION_TOPIC_CONFIG2 =
|
||||
new RepartitionTopicConfig(REPARTITION_TOPIC_NAME2, TOPIC_CONFIG2, 2, true);
|
||||
private static final TopicsInfo TOPICS_INFO1 = new TopicsInfo(
|
||||
mkSet(REPARTITION_TOPIC_NAME1),
|
||||
mkSet(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2),
|
||||
Set.of(REPARTITION_TOPIC_NAME1),
|
||||
Set.of(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2),
|
||||
mkMap(
|
||||
mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1),
|
||||
mkEntry(REPARTITION_TOPIC_NAME2, REPARTITION_TOPIC_CONFIG2)
|
||||
|
@ -91,8 +91,8 @@ public class RepartitionTopicsTest {
|
|||
Collections.emptyMap()
|
||||
);
|
||||
private static final TopicsInfo TOPICS_INFO2 = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME1),
|
||||
mkSet(REPARTITION_TOPIC_NAME1),
|
||||
Set.of(SINK_TOPIC_NAME1),
|
||||
Set.of(REPARTITION_TOPIC_NAME1),
|
||||
mkMap(mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1)),
|
||||
Collections.emptyMap()
|
||||
);
|
||||
|
@ -117,8 +117,8 @@ public class RepartitionTopicsTest {
|
|||
public void shouldSetupRepartitionTopics() {
|
||||
when(internalTopologyBuilder.subtopologyToTopicsInfo())
|
||||
.thenReturn(mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1), mkEntry(SUBTOPOLOGY_1, TOPICS_INFO2)));
|
||||
final Set<String> coPartitionGroup1 = mkSet(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2);
|
||||
final Set<String> coPartitionGroup2 = mkSet(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_NAME2);
|
||||
final Set<String> coPartitionGroup1 = Set.of(SOURCE_TOPIC_NAME1, SOURCE_TOPIC_NAME2);
|
||||
final Set<String> coPartitionGroup2 = Set.of(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_NAME2);
|
||||
final List<Set<String>> coPartitionGroups = Arrays.asList(coPartitionGroup1, coPartitionGroup2);
|
||||
when(internalTopologyBuilder.copartitionGroups()).thenReturn(coPartitionGroups);
|
||||
when(internalTopicManager.makeReady(
|
||||
|
@ -156,7 +156,7 @@ public class RepartitionTopicsTest {
|
|||
|
||||
@Test
|
||||
public void shouldReturnMissingSourceTopics() {
|
||||
final Set<String> missingSourceTopics = mkSet(SOURCE_TOPIC_NAME1);
|
||||
final Set<String> missingSourceTopics = Set.of(SOURCE_TOPIC_NAME1);
|
||||
when(internalTopologyBuilder.subtopologyToTopicsInfo())
|
||||
.thenReturn(mkMap(mkEntry(SUBTOPOLOGY_0, TOPICS_INFO1), mkEntry(SUBTOPOLOGY_1, TOPICS_INFO2)));
|
||||
setupClusterWithMissingTopics(missingSourceTopics, false);
|
||||
|
@ -208,8 +208,8 @@ public class RepartitionTopicsTest {
|
|||
final RepartitionTopicConfig repartitionTopicConfigWithoutPartitionCount =
|
||||
new RepartitionTopicConfig(REPARTITION_WITHOUT_PARTITION_COUNT, TOPIC_CONFIG5);
|
||||
final TopicsInfo topicsInfo = new TopicsInfo(
|
||||
mkSet(REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
mkSet(SOURCE_TOPIC_NAME1),
|
||||
Set.of(REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
Set.of(SOURCE_TOPIC_NAME1),
|
||||
mkMap(
|
||||
mkEntry(REPARTITION_WITHOUT_PARTITION_COUNT, repartitionTopicConfigWithoutPartitionCount)
|
||||
),
|
||||
|
@ -220,7 +220,7 @@ public class RepartitionTopicsTest {
|
|||
mkEntry(SUBTOPOLOGY_0, topicsInfo),
|
||||
mkEntry(SUBTOPOLOGY_1, setupTopicInfoWithRepartitionTopicWithoutPartitionCount(repartitionTopicConfigWithoutPartitionCount))
|
||||
));
|
||||
setupClusterWithMissingPartitionCounts(mkSet(SOURCE_TOPIC_NAME1), true);
|
||||
setupClusterWithMissingPartitionCounts(Set.of(SOURCE_TOPIC_NAME1), true);
|
||||
final RepartitionTopics repartitionTopics = new RepartitionTopics(
|
||||
new TopologyMetadata(internalTopologyBuilder, config),
|
||||
internalTopicManager,
|
||||
|
@ -243,8 +243,8 @@ public class RepartitionTopicsTest {
|
|||
final RepartitionTopicConfig repartitionTopicConfigWithoutPartitionCount =
|
||||
new RepartitionTopicConfig(REPARTITION_WITHOUT_PARTITION_COUNT, TOPIC_CONFIG5);
|
||||
final TopicsInfo topicsInfo = new TopicsInfo(
|
||||
mkSet(REPARTITION_TOPIC_NAME1, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
mkSet(SOURCE_TOPIC_NAME1, REPARTITION_TOPIC_NAME2),
|
||||
Set.of(REPARTITION_TOPIC_NAME1, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
Set.of(SOURCE_TOPIC_NAME1, REPARTITION_TOPIC_NAME2),
|
||||
mkMap(
|
||||
mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1),
|
||||
mkEntry(REPARTITION_TOPIC_NAME2, REPARTITION_TOPIC_CONFIG2),
|
||||
|
@ -297,8 +297,8 @@ public class RepartitionTopicsTest {
|
|||
final RepartitionTopicConfig repartitionTopicConfigWithoutPartitionCount =
|
||||
new RepartitionTopicConfig(REPARTITION_WITHOUT_PARTITION_COUNT, TOPIC_CONFIG5);
|
||||
final TopicsInfo topicsInfo = new TopicsInfo(
|
||||
mkSet(REPARTITION_TOPIC_NAME2, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
mkSet(SOURCE_TOPIC_NAME1, REPARTITION_TOPIC_NAME1),
|
||||
Set.of(REPARTITION_TOPIC_NAME2, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
Set.of(SOURCE_TOPIC_NAME1, REPARTITION_TOPIC_NAME1),
|
||||
mkMap(
|
||||
mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1),
|
||||
mkEntry(REPARTITION_TOPIC_NAME2, REPARTITION_TOPIC_CONFIG2),
|
||||
|
@ -350,8 +350,8 @@ public class RepartitionTopicsTest {
|
|||
@Test
|
||||
public void shouldNotSetupRepartitionTopicsWhenTopologyDoesNotContainAnyRepartitionTopics() {
|
||||
final TopicsInfo topicsInfo = new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME1),
|
||||
mkSet(SOURCE_TOPIC_NAME1),
|
||||
Set.of(SINK_TOPIC_NAME1),
|
||||
Set.of(SOURCE_TOPIC_NAME1),
|
||||
Collections.emptyMap(),
|
||||
Collections.emptyMap()
|
||||
);
|
||||
|
@ -406,7 +406,7 @@ public class RepartitionTopicsTest {
|
|||
private void setupClusterWithMissingTopicsAndMissingPartitionCounts(final Set<String> missingTopics,
|
||||
final Set<String> topicsWithMissingPartitionCounts,
|
||||
final boolean mockPartitionCount) {
|
||||
final Set<String> topics = mkSet(
|
||||
final Set<String> topics = new HashSet<>(List.of(
|
||||
SOURCE_TOPIC_NAME1,
|
||||
SOURCE_TOPIC_NAME2,
|
||||
SOURCE_TOPIC_NAME3,
|
||||
|
@ -417,7 +417,7 @@ public class RepartitionTopicsTest {
|
|||
REPARTITION_TOPIC_NAME3,
|
||||
REPARTITION_TOPIC_NAME4,
|
||||
SOME_OTHER_TOPIC
|
||||
);
|
||||
));
|
||||
topics.removeAll(missingTopics);
|
||||
when(clusterMetadata.topics()).thenReturn(topics);
|
||||
if (mockPartitionCount) {
|
||||
|
@ -428,8 +428,8 @@ public class RepartitionTopicsTest {
|
|||
|
||||
private TopicsInfo setupTopicInfoWithRepartitionTopicWithoutPartitionCount(final RepartitionTopicConfig repartitionTopicConfigWithoutPartitionCount) {
|
||||
return new TopicsInfo(
|
||||
mkSet(SINK_TOPIC_NAME2),
|
||||
mkSet(REPARTITION_TOPIC_NAME1, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
Set.of(SINK_TOPIC_NAME2),
|
||||
Set.of(REPARTITION_TOPIC_NAME1, REPARTITION_WITHOUT_PARTITION_COUNT),
|
||||
mkMap(
|
||||
mkEntry(REPARTITION_TOPIC_NAME1, REPARTITION_TOPIC_CONFIG1),
|
||||
mkEntry(REPARTITION_WITHOUT_PARTITION_COUNT, repartitionTopicConfigWithoutPartitionCount)
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue