MINOR: Cleanup simplify set initialization with Set.of (#19925)

Simplify Set initialization and reduce the overhead of creating extra
collections.

The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())

This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Jhen-Yung Hsu 2025-06-11 18:36:14 +08:00 committed by GitHub
parent b704280a0a
commit 2e968560e0
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
115 changed files with 544 additions and 635 deletions

View File

@ -17,8 +17,6 @@
package org.apache.kafka.common.config;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
/**
@ -64,8 +62,8 @@ public class LogLevelConfig {
*/
public static final String TRACE_LOG_LEVEL = "TRACE";
public static final Set<String> VALID_LOG_LEVELS = new HashSet<>(Arrays.asList(
public static final Set<String> VALID_LOG_LEVELS = Set.of(
FATAL_LOG_LEVEL, ERROR_LOG_LEVEL, WARN_LOG_LEVEL,
INFO_LOG_LEVEL, DEBUG_LOG_LEVEL, TRACE_LOG_LEVEL
));
);
}

View File

@ -957,9 +957,9 @@ public class MetadataTest {
Cluster cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), oldClusterId);
assertEquals(cluster.nodes().size(), oldNodes);
assertEquals(cluster.invalidTopics(), new HashSet<>(Arrays.asList("oldInvalidTopic", "keepInvalidTopic")));
assertEquals(cluster.unauthorizedTopics(), new HashSet<>(Arrays.asList("oldUnauthorizedTopic", "keepUnauthorizedTopic")));
assertEquals(cluster.topics(), new HashSet<>(Arrays.asList("oldValidTopic", "keepValidTopic")));
assertEquals(cluster.invalidTopics(), Set.of("oldInvalidTopic", "keepInvalidTopic"));
assertEquals(cluster.unauthorizedTopics(), Set.of("oldUnauthorizedTopic", "keepUnauthorizedTopic"));
assertEquals(cluster.topics(), Set.of("oldValidTopic", "keepValidTopic"));
assertEquals(cluster.partitionsForTopic("oldValidTopic").size(), 2);
assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 3);
assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values()));
@ -992,9 +992,9 @@ public class MetadataTest {
cluster = metadata.fetch();
assertEquals(cluster.clusterResource().clusterId(), newClusterId);
assertEquals(cluster.nodes().size(), newNodes);
assertEquals(cluster.invalidTopics(), new HashSet<>(Arrays.asList("keepInvalidTopic", "newInvalidTopic")));
assertEquals(cluster.unauthorizedTopics(), new HashSet<>(Arrays.asList("keepUnauthorizedTopic", "newUnauthorizedTopic")));
assertEquals(cluster.topics(), new HashSet<>(Arrays.asList("keepValidTopic", "newValidTopic")));
assertEquals(cluster.invalidTopics(), Set.of("keepInvalidTopic", "newInvalidTopic"));
assertEquals(cluster.unauthorizedTopics(), Set.of("keepUnauthorizedTopic", "newUnauthorizedTopic"));
assertEquals(cluster.topics(), Set.of("keepValidTopic", "newValidTopic"));
assertEquals(cluster.partitionsForTopic("keepValidTopic").size(), 2);
assertEquals(cluster.partitionsForTopic("newValidTopic").size(), 4);
assertEquals(new HashSet<>(cluster.topicIds()), new HashSet<>(topicIds.values()));

View File

@ -390,10 +390,10 @@ public class KafkaAdminClientTest {
assertNull(cluster.controller());
}
assertEquals("Ek8tjqq1QBWfnaoyHFZqDg", cluster.clusterResource().clusterId());
assertEquals(new HashSet<>(asList(
assertEquals(Set.of(
new Node(0, "controller0.com", 9092),
new Node(1, "controller1.com", 9092),
new Node(2, "controller2.com", 9092))), new HashSet<>(cluster.nodes()));
new Node(2, "controller2.com", 9092)), new HashSet<>(cluster.nodes()));
}
@Test
@ -1592,7 +1592,7 @@ public class KafkaAdminClientTest {
Map<String, TopicDescription> topicDescriptions = result.allTopicNames().get();
TopicDescription topicDescription = topicDescriptions.get(topicName0);
assertEquals(new HashSet<>(asList(AclOperation.DESCRIBE, AclOperation.ALTER)),
assertEquals(Set.of(AclOperation.DESCRIBE, AclOperation.ALTER),
topicDescription.authorizedOperations());
}
}
@ -2084,7 +2084,7 @@ public class KafkaAdminClientTest {
electionResults, ApiKeys.ELECT_LEADERS.latestVersion()));
ElectLeadersResult results = env.adminClient().electLeaders(
electionType,
new HashSet<>(asList(topic1, topic2)));
Set.of(topic1, topic2));
assertEquals(ClusterAuthorizationException.class, results.partitions().get().get(topic2).get().getClass());
// Test a call where there are no errors. By mutating the internal of election results
@ -2096,14 +2096,14 @@ public class KafkaAdminClientTest {
env.kafkaClient().prepareResponse(new ElectLeadersResponse(0, Errors.NONE.code(), electionResults,
ApiKeys.ELECT_LEADERS.latestVersion()));
results = env.adminClient().electLeaders(electionType, new HashSet<>(asList(topic1, topic2)));
results = env.adminClient().electLeaders(electionType, Set.of(topic1, topic2));
assertFalse(results.partitions().get().get(topic1).isPresent());
assertFalse(results.partitions().get().get(topic2).isPresent());
// Now try a timeout
results = env.adminClient().electLeaders(
electionType,
new HashSet<>(asList(topic1, topic2)),
Set.of(topic1, topic2),
new ElectLeadersOptions().timeoutMs(100));
TestUtils.assertFutureThrows(TimeoutException.class, results.partitions());
}
@ -2127,7 +2127,7 @@ public class KafkaAdminClientTest {
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(asList(
broker0Resource,
broker1Resource)).values();
assertEquals(new HashSet<>(asList(broker0Resource, broker1Resource)), result.keySet());
assertEquals(Set.of(broker0Resource, broker1Resource), result.keySet());
result.get(broker0Resource).get();
result.get(broker1Resource).get();
}
@ -2149,7 +2149,7 @@ public class KafkaAdminClientTest {
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(asList(
brokerResource,
brokerLoggerResource)).values();
assertEquals(new HashSet<>(asList(brokerResource, brokerLoggerResource)), result.keySet());
assertEquals(Set.of(brokerResource, brokerLoggerResource), result.keySet());
result.get(brokerResource).get();
result.get(brokerLoggerResource).get();
}
@ -2168,7 +2168,7 @@ public class KafkaAdminClientTest {
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(asList(
topic,
topic2)).values();
assertEquals(new HashSet<>(asList(topic, topic2)), result.keySet());
assertEquals(Set.of(topic, topic2), result.keySet());
result.get(topic);
TestUtils.assertFutureThrows(ApiException.class, result.get(topic2));
}
@ -2189,7 +2189,7 @@ public class KafkaAdminClientTest {
.setConfigs(emptyList())))));
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(singletonList(
topic)).values();
assertEquals(new HashSet<>(singletonList(topic)), result.keySet());
assertEquals(Set.of(topic), result.keySet());
assertNotNull(result.get(topic).get());
assertNull(result.get(unrequested));
}
@ -2212,7 +2212,7 @@ public class KafkaAdminClientTest {
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(asList(
resource,
resource1)).values();
assertEquals(new HashSet<>(asList(resource, resource1)), result.keySet());
assertEquals(Set.of(resource, resource1), result.keySet());
assertNotNull(result.get(resource).get());
assertNotNull(result.get(resource1).get());
}
@ -2239,7 +2239,7 @@ public class KafkaAdminClientTest {
Map<ConfigResource, KafkaFuture<Config>> result = env.adminClient().describeConfigs(asList(
resource1,
resource2)).values();
assertEquals(new HashSet<>(asList(resource1, resource2)), result.keySet());
assertEquals(Set.of(resource1, resource2), result.keySet());
assertNotNull(result.get(resource1).get());
assertNotNull(result.get(resource2).get());
}
@ -2905,7 +2905,7 @@ public class KafkaAdminClientTest {
assertEquals(env.cluster().clusterResource().clusterId(), result2.clusterId().get());
assertEquals(new HashSet<>(env.cluster().nodes()), new HashSet<>(result2.nodes().get()));
assertEquals(3, result2.controller().get().id());
assertEquals(new HashSet<>(asList(AclOperation.DESCRIBE, AclOperation.ALTER)),
assertEquals(Set.of(AclOperation.DESCRIBE, AclOperation.ALTER),
result2.authorizedOperations().get());
}
}
@ -7922,7 +7922,7 @@ public class KafkaAdminClientTest {
.setErrorMessage(Errors.UNKNOWN_TOPIC_OR_PARTITION.message());
env.kafkaClient().prepareResponse(new ListPartitionReassignmentsResponse(unknownTpData));
ListPartitionReassignmentsResult unknownTpResult = env.adminClient().listPartitionReassignments(new HashSet<>(asList(tp1, tp2)));
ListPartitionReassignmentsResult unknownTpResult = env.adminClient().listPartitionReassignments(Set.of(tp1, tp2));
TestUtils.assertFutureThrows(UnknownTopicOrPartitionException.class, unknownTpResult.reassignments());
// 3. Success

View File

@ -20,8 +20,6 @@ import org.apache.kafka.common.ConsumerGroupState;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -29,7 +27,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
public class ListConsumerGroupsOptionsTest {
@Test
public void testState() {
Set<ConsumerGroupState> consumerGroupStates = new HashSet<>(Arrays.asList(ConsumerGroupState.values()));
Set<ConsumerGroupState> consumerGroupStates = Set.of(ConsumerGroupState.values());
ListConsumerGroupsOptions options = new ListConsumerGroupsOptions().inStates(consumerGroupStates);
assertEquals(consumerGroupStates, options.states());
}

View File

@ -40,7 +40,6 @@ import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -397,7 +396,7 @@ class AdminApiDriverTest {
public void testRetryLookupAndDisableBatchAfterNoBatchedFindCoordinatorsException() {
MockTime time = new MockTime();
LogContext lc = new LogContext();
Set<String> groupIds = new HashSet<>(Arrays.asList("g1", "g2"));
Set<String> groupIds = Set.of("g1", "g2");
DeleteConsumerGroupsHandler handler = new DeleteConsumerGroupsHandler(lc);
AdminApiFuture<CoordinatorKey, Void> future = AdminApiFuture.forKeys(
groupIds.stream().map(CoordinatorKey::byGroupId).collect(Collectors.toSet()));

View File

@ -55,9 +55,9 @@ public class CoordinatorStrategyTest {
@Test
public void testBuildLookupRequest() {
CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext());
FindCoordinatorRequest.Builder request = strategy.buildRequest(new HashSet<>(Arrays.asList(
FindCoordinatorRequest.Builder request = strategy.buildRequest(Set.of(
CoordinatorKey.byGroupId("foo"),
CoordinatorKey.byGroupId("bar"))));
CoordinatorKey.byGroupId("bar")));
assertEquals("", request.data().key());
assertEquals(2, request.data().coordinatorKeys().size());
assertEquals(CoordinatorType.GROUP, CoordinatorType.forId(request.data().keyType()));
@ -67,8 +67,8 @@ public class CoordinatorStrategyTest {
public void testBuildLookupRequestNonRepresentable() {
CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext());
FindCoordinatorRequest.Builder request = strategy.buildRequest(new HashSet<>(Arrays.asList(
CoordinatorKey.byGroupId("foo"),
null)));
CoordinatorKey.byGroupId("foo"),
null)));
assertEquals("", request.data().key());
assertEquals(1, request.data().coordinatorKeys().size());
}
@ -90,7 +90,7 @@ public class CoordinatorStrategyTest {
strategy.disableBatch();
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(
new HashSet<>(Collections.singletonList(CoordinatorKey.byTransactionalId("txnid")))));
Set.of(CoordinatorKey.byTransactionalId("txnid"))));
}
@Test
@ -105,9 +105,9 @@ public class CoordinatorStrategyTest {
CoordinatorStrategy strategy = new CoordinatorStrategy(CoordinatorType.GROUP, new LogContext());
assertThrows(IllegalArgumentException.class, () -> strategy.buildRequest(
new HashSet<>(Arrays.asList(
CoordinatorKey.byGroupId("group"),
CoordinatorKey.byTransactionalId("txnid")))));
Set.of(
CoordinatorKey.byGroupId("group"),
CoordinatorKey.byTransactionalId("txnid"))));
}
@Test
@ -161,7 +161,7 @@ public class CoordinatorStrategyTest {
.setPort(9092)
.setNodeId(2)));
AdminApiLookupStrategy.LookupResult<CoordinatorKey> result = runLookup(new HashSet<>(Arrays.asList(group1, group2)), responseData);
AdminApiLookupStrategy.LookupResult<CoordinatorKey> result = runLookup(Set.of(group1, group2), responseData);
Map<CoordinatorKey, Integer> expectedResult = new HashMap<>();
expectedResult.put(group1, 1);
expectedResult.put(group2, 2);
@ -204,7 +204,7 @@ public class CoordinatorStrategyTest {
.setHost("localhost")
.setPort(9092)
.setNodeId(2)));
AdminApiLookupStrategy.LookupResult<CoordinatorKey> result = runLookup(new HashSet<>(Arrays.asList(group1, group2)), responseData);
AdminApiLookupStrategy.LookupResult<CoordinatorKey> result = runLookup(Set.of(group1, group2), responseData);
assertEquals(emptyMap(), result.failedKeys);
assertEquals(singletonMap(group2, 2), result.mappedKeys);

View File

@ -34,10 +34,8 @@ import org.apache.kafka.common.utils.LogContext;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -55,7 +53,7 @@ public class DeleteConsumerGroupOffsetsHandlerTest {
private final TopicPartition t0p0 = new TopicPartition("t0", 0);
private final TopicPartition t0p1 = new TopicPartition("t0", 1);
private final TopicPartition t1p0 = new TopicPartition("t1", 0);
private final Set<TopicPartition> tps = new HashSet<>(Arrays.asList(t0p0, t0p1, t1p0));
private final Set<TopicPartition> tps = Set.of(t0p0, t0p1, t1p0);
@Test
public void testBuildRequest() {

View File

@ -53,7 +53,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
@ -82,10 +81,10 @@ public class DescribeConsumerGroupsHandlerTest {
CoordinatorKey.byGroupId(groupId2)
));
private final Node coordinator = new Node(1, "host", 1234);
private final Set<TopicPartition> tps = new HashSet<>(Arrays.asList(
private final Set<TopicPartition> tps = Set.of(
new TopicPartition("foo", 0),
new TopicPartition("bar", 1)
));
);
@ParameterizedTest
@ValueSource(booleans = {true, false})

View File

@ -333,7 +333,7 @@ public abstract class ConsumerCoordinatorTest {
List<Collection<String>> capturedTopics = topicsCaptor.getAllValues();
// expected the final group subscribed topics to be updated to "topic1" and "topic2"
Set<String> expectedTopicsGotCalled = new HashSet<>(Arrays.asList(topic1, topic2));
Set<String> expectedTopicsGotCalled = Set.of(topic1, topic2);
assertEquals(expectedTopicsGotCalled, capturedTopics.get(1));
}
}
@ -1279,7 +1279,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.poll(time.timer(Long.MAX_VALUE));
// Make sure that the metadata was refreshed during the rebalance and thus subscriptions now contain two topics.
final Set<String> updatedSubscriptionSet = new HashSet<>(Arrays.asList(topic1, topic2));
final Set<String> updatedSubscriptionSet = Set.of(topic1, topic2);
assertEquals(updatedSubscriptionSet, subscriptions.subscription());
// Refresh the metadata again. Since there have been no changes since the last refresh, it won't trigger
@ -1300,7 +1300,7 @@ public abstract class ConsumerCoordinatorTest {
}
}));
coordinator.maybeUpdateSubscriptionMetadata();
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), subscriptions.subscription());
assertEquals(Set.of(topic1, topic2), subscriptions.subscription());
client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE));
coordinator.ensureCoordinatorReady(time.timer(Long.MAX_VALUE));
@ -2072,7 +2072,7 @@ public abstract class ConsumerCoordinatorTest {
coordinator.poll(time.timer(Long.MAX_VALUE));
assertFalse(coordinator.rejoinNeededOrPending());
assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions());
assertEquals(Set.of(tp1, tp2), subscriptions.assignedPartitions());
}
/**
@ -2264,7 +2264,7 @@ public abstract class ConsumerCoordinatorTest {
// and join the group again
rebalanceListener.revoked = null;
rebalanceListener.assigned = null;
subscriptions.subscribe(new HashSet<>(Arrays.asList(topic1, otherTopic)), Optional.of(rebalanceListener));
subscriptions.subscribe(Set.of(topic1, otherTopic), Optional.of(rebalanceListener));
client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE));
client.prepareResponse(syncGroupResponse(assigned, Errors.NONE));
coordinator.joinGroupIfNeeded(time.timer(Long.MAX_VALUE));

View File

@ -32,7 +32,6 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
@ -204,6 +203,6 @@ public class FetchBufferTest {
* This is a handy utility method for returning a set from a varargs array.
*/
private static Set<TopicPartition> partitions(TopicPartition... partitions) {
return new HashSet<>(Arrays.asList(partitions));
return Set.of(partitions);
}
}

View File

@ -53,7 +53,6 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Properties;
@ -721,7 +720,7 @@ public class FetchCollectorTest {
* This is a handy utility method for returning a set from a varargs array.
*/
private static Set<TopicPartition> partitions(TopicPartition... partitions) {
return new HashSet<>(Arrays.asList(partitions));
return Set.of(partitions);
}
private void buildDependencies() {

View File

@ -2479,7 +2479,7 @@ public class FetchRequestManagerTest {
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
List<ConsumerRecord<byte[], byte[]>> fetchedConsumerRecords = fetchedRecords.get(tp0);
Set<String> expectedCommittedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2"));
Set<String> expectedCommittedKeys = Set.of("commit1-1", "commit1-2");
Set<String> actuallyCommittedKeys = new HashSet<>();
for (ConsumerRecord<byte[], byte[]> consumerRecord : fetchedConsumerRecords) {
actuallyCommittedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
@ -2741,7 +2741,7 @@ public class FetchRequestManagerTest {
public void testConsumingViaIncrementalFetchRequests() {
buildFetcher(2);
assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
assignFromUser(Set.of(tp0, tp1));
subscriptions.seekValidated(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), metadata.currentLeader(tp0)));
subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1)));
@ -3196,7 +3196,7 @@ public class FetchRequestManagerTest {
// Setup so that tp0 & tp1 are subscribed and will be fetched from.
// Also, setup client's metadata for tp0 & tp1.
subscriptions.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
subscriptions.assignFromUser(Set.of(tp0, tp1));
client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4),
tp -> validLeaderEpoch, topicIds, false));
@ -3289,7 +3289,7 @@ public class FetchRequestManagerTest {
// Setup so that tp0 & tp1 are subscribed and will be fetched from.
// Also, setup client's metadata for tp0 & tp1.
subscriptions.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
subscriptions.assignFromUser(Set.of(tp0, tp1));
client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4),
tp -> validLeaderEpoch, topicIds, false));

View File

@ -2466,7 +2466,7 @@ public class FetcherTest {
assertTrue(fetchedRecords.containsKey(tp0));
assertEquals(fetchedRecords.get(tp0).size(), 2);
List<ConsumerRecord<byte[], byte[]>> fetchedConsumerRecords = fetchedRecords.get(tp0);
Set<String> expectedCommittedKeys = new HashSet<>(Arrays.asList("commit1-1", "commit1-2"));
Set<String> expectedCommittedKeys = Set.of("commit1-1", "commit1-2");
Set<String> actuallyCommittedKeys = new HashSet<>();
for (ConsumerRecord<byte[], byte[]> consumerRecord : fetchedConsumerRecords) {
actuallyCommittedKeys.add(new String(consumerRecord.key(), StandardCharsets.UTF_8));
@ -2728,7 +2728,7 @@ public class FetcherTest {
public void testConsumingViaIncrementalFetchRequests() {
buildFetcher(2);
assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
assignFromUser(Set.of(tp0, tp1));
subscriptions.seekValidated(tp0, new SubscriptionState.FetchPosition(0, Optional.empty(), metadata.currentLeader(tp0)));
subscriptions.seekValidated(tp1, new SubscriptionState.FetchPosition(1, Optional.empty(), metadata.currentLeader(tp1)));
@ -3473,7 +3473,7 @@ public class FetcherTest {
// Setup so that tp0 & tp1 are subscribed and will be fetched from.
// Also, setup client's metadata for tp0 & tp1.
subscriptions.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
subscriptions.assignFromUser(Set.of(tp0, tp1));
client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4),
tp -> validLeaderEpoch, topicIds, false));
@ -3566,7 +3566,7 @@ public class FetcherTest {
// Setup so that tp0 & tp1 are subscribed and will be fetched from.
// Also, setup client's metadata for tp0 & tp1.
subscriptions.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
subscriptions.assignFromUser(Set.of(tp0, tp1));
client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(2, singletonMap(topicName, 4),
tp -> validLeaderEpoch, topicIds, false));

View File

@ -51,7 +51,6 @@ import org.junit.jupiter.params.provider.MethodSource;
import org.mockito.ArgumentCaptor;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -774,7 +773,7 @@ public class OffsetsRequestManagerTest {
// tp2 added to the assignment when the Offset Fetch request is already sent including tp1 only
TopicPartition tp2 = new TopicPartition("topic2", 2);
Set<TopicPartition> initPartitions2 = new HashSet<>(Arrays.asList(tp1, tp2));
Set<TopicPartition> initPartitions2 = Set.of(tp1, tp2);
mockAssignedPartitionsMissingPositions(initPartitions2, initPartitions2, leaderAndEpoch);
// tp2 requires a position, but shouldn't be reset after receiving the offset fetch response that will only

View File

@ -33,8 +33,6 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
@ -181,6 +179,6 @@ public class ShareFetchBufferTest {
* This is a handy utility method for returning a set from a varargs array.
*/
private static Set<TopicIdPartition> partitions(TopicIdPartition... partitions) {
return new HashSet<>(Arrays.asList(partitions));
return Set.of(partitions);
}
}

View File

@ -90,9 +90,9 @@ public class StreamsRebalanceDataTest {
@Test
public void assignmentShouldNotBeModifiable() {
final StreamsRebalanceData.Assignment assignment = new StreamsRebalanceData.Assignment(
new HashSet<>(Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 1))),
new HashSet<>(Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 2))),
new HashSet<>(Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 3)))
Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 1)),
Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 2)),
Set.of(new StreamsRebalanceData.TaskId("subtopologyId1", 3))
);
assertThrows(
@ -220,8 +220,8 @@ public class StreamsRebalanceDataTest {
@Test
public void subtopologyShouldNotBeModifiable() {
final StreamsRebalanceData.Subtopology subtopology = new StreamsRebalanceData.Subtopology(
new HashSet<>(Set.of("sourceTopic1")),
new HashSet<>(Set.of("repartitionSinkTopic1")),
Set.of("sourceTopic1"),
Set.of("repartitionSinkTopic1"),
Map.of("repartitionSourceTopic1", new StreamsRebalanceData.TopicInfo(Optional.of(1), Optional.of((short) 1), Map.of()))
.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)),

View File

@ -33,10 +33,8 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import java.util.function.LongSupplier;
@ -80,7 +78,7 @@ public class SubscriptionStateTest {
@Test
public void partitionAssignmentChangeOnTopicSubscription() {
state.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
state.assignFromUser(Set.of(tp0, tp1));
// assigned partitions should immediately change
assertEquals(2, state.assignedPartitions().size());
assertEquals(2, state.numAssignedPartitions());
@ -394,7 +392,7 @@ public class SubscriptionStateTest {
@Test
public void patternSubscription() {
state.subscribe(Pattern.compile(".*"), Optional.of(rebalanceListener));
state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1)));
state.subscribeFromPattern(Set.of(topic, topic1));
assertEquals(2, state.subscription().size(), "Expected subscribed topics count is incorrect");
}
@ -434,7 +432,7 @@ public class SubscriptionStateTest {
@Test
public void unsubscribeUserAssignment() {
state.assignFromUser(new HashSet<>(Arrays.asList(tp0, tp1)));
state.assignFromUser(Set.of(tp0, tp1));
state.unsubscribe();
state.subscribe(singleton(topic), Optional.of(rebalanceListener));
assertEquals(singleton(topic), state.subscription());
@ -452,7 +450,7 @@ public class SubscriptionStateTest {
@Test
public void unsubscription() {
state.subscribe(Pattern.compile(".*"), Optional.of(rebalanceListener));
state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1)));
state.subscribeFromPattern(Set.of(topic, topic1));
assertTrue(state.checkAssignmentMatchedSubscription(singleton(tp1)));
state.assignFromSubscribed(singleton(tp1));

View File

@ -22,8 +22,7 @@ import org.apache.kafka.common.metrics.Metrics;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import static org.apache.kafka.clients.consumer.internals.ConsumerUtils.CONSUMER_METRIC_GROUP;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -48,14 +47,14 @@ public class AsyncConsumerMetricsTest {
public void shouldMetricNames() {
// create
consumerMetrics = new AsyncConsumerMetrics(metrics);
HashSet<MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
Set<MetricName> expectedMetrics = Set.of(
metrics.metricName("last-poll-seconds-ago", CONSUMER_METRIC_GROUP),
metrics.metricName("time-between-poll-avg", CONSUMER_METRIC_GROUP),
metrics.metricName("time-between-poll-max", CONSUMER_METRIC_GROUP),
metrics.metricName("poll-idle-ratio-avg", CONSUMER_METRIC_GROUP),
metrics.metricName("commit-sync-time-ns-total", CONSUMER_METRIC_GROUP),
metrics.metricName("committed-time-ns-total", CONSUMER_METRIC_GROUP)
));
);
expectedMetrics.forEach(
metricName -> assertTrue(
metrics.metrics().containsKey(metricName),
@ -63,7 +62,7 @@ public class AsyncConsumerMetricsTest {
)
);
HashSet<MetricName> expectedConsumerMetrics = new HashSet<>(Arrays.asList(
Set<MetricName> expectedConsumerMetrics = Set.of(
metrics.metricName("time-between-network-thread-poll-avg", CONSUMER_METRIC_GROUP),
metrics.metricName("time-between-network-thread-poll-max", CONSUMER_METRIC_GROUP),
metrics.metricName("application-event-queue-size", CONSUMER_METRIC_GROUP),
@ -79,7 +78,7 @@ public class AsyncConsumerMetricsTest {
metrics.metricName("background-event-queue-time-max", CONSUMER_METRIC_GROUP),
metrics.metricName("background-event-queue-processing-time-avg", CONSUMER_METRIC_GROUP),
metrics.metricName("background-event-queue-processing-time-max", CONSUMER_METRIC_GROUP)
));
);
expectedConsumerMetrics.forEach(
metricName -> assertTrue(
metrics.metrics().containsKey(metricName),

View File

@ -29,10 +29,8 @@ import org.apache.kafka.common.utils.Time;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
@ -257,14 +255,14 @@ public class ProducerMetadataTest {
assertTrue(metadata.updateRequested());
assertEquals(0, metadata.timeToNextUpdate(now));
assertEquals(metadata.topics(), new HashSet<>(Arrays.asList(topic1, topic2, topic3)));
assertEquals(metadata.newTopics(), new HashSet<>(Arrays.asList(topic2, topic3)));
assertEquals(metadata.topics(), Set.of(topic1, topic2, topic3));
assertEquals(metadata.newTopics(), Set.of(topic2, topic3));
// Perform the partial update for a subset of the new topics.
now += 1000;
assertTrue(metadata.updateRequested());
metadata.updateWithCurrentRequestVersion(responseWithTopics(Collections.singleton(topic2)), true, now);
assertEquals(metadata.topics(), new HashSet<>(Arrays.asList(topic1, topic2, topic3)));
assertEquals(metadata.topics(), Set.of(topic1, topic2, topic3));
assertEquals(metadata.newTopics(), Collections.singleton(topic3));
}
@ -302,7 +300,7 @@ public class ProducerMetadataTest {
// Perform the full update. This should clear the update request.
now += 1000;
metadata.updateWithCurrentRequestVersion(responseWithTopics(new HashSet<>(Arrays.asList(topic1, topic2))), false, now);
metadata.updateWithCurrentRequestVersion(responseWithTopics(Set.of(topic1, topic2)), false, now);
assertFalse(metadata.updateRequested());
}

View File

@ -59,7 +59,6 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Deque;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -167,7 +166,7 @@ public class RecordAccumulatorTest {
accum.append(topic, partition4, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, time.milliseconds(), cluster);
// drain batches from 2 nodes: node1 => tp1, node2 => tp3, because the max request size is full after the first batch drained
Map<Integer, List<ProducerBatch>> batches1 = accum.drain(metadataCache, new HashSet<>(Arrays.asList(node1, node2)), (int) batchSize, 0);
Map<Integer, List<ProducerBatch>> batches1 = accum.drain(metadataCache, Set.of(node1, node2), (int) batchSize, 0);
verifyTopicPartitionInBatches(batches1, tp1, tp3);
// add record for tp1, tp3
@ -176,11 +175,11 @@ public class RecordAccumulatorTest {
// drain batches from 2 nodes: node1 => tp2, node2 => tp4, because the max request size is full after the first batch drained
// The drain index should start from next topic partition, that is, node1 => tp2, node2 => tp4
Map<Integer, List<ProducerBatch>> batches2 = accum.drain(metadataCache, new HashSet<>(Arrays.asList(node1, node2)), (int) batchSize, 0);
Map<Integer, List<ProducerBatch>> batches2 = accum.drain(metadataCache, Set.of(node1, node2), (int) batchSize, 0);
verifyTopicPartitionInBatches(batches2, tp2, tp4);
// make sure in next run, the drain index will start from the beginning
Map<Integer, List<ProducerBatch>> batches3 = accum.drain(metadataCache, new HashSet<>(Arrays.asList(node1, node2)), (int) batchSize, 0);
Map<Integer, List<ProducerBatch>> batches3 = accum.drain(metadataCache, Set.of(node1, node2), (int) batchSize, 0);
verifyTopicPartitionInBatches(batches3, tp1, tp3);
// add record for tp2, tp3, tp4 and mute the tp4
@ -189,7 +188,7 @@ public class RecordAccumulatorTest {
accum.append(topic, partition4, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, time.milliseconds(), cluster);
accum.mutePartition(tp4);
// drain batches from 2 nodes: node1 => tp2, node2 => tp3 (because tp4 is muted)
Map<Integer, List<ProducerBatch>> batches4 = accum.drain(metadataCache, new HashSet<>(Arrays.asList(node1, node2)), (int) batchSize, 0);
Map<Integer, List<ProducerBatch>> batches4 = accum.drain(metadataCache, Set.of(node1, node2), (int) batchSize, 0);
verifyTopicPartitionInBatches(batches4, tp2, tp3);
// add record for tp1, tp2, tp3, and unmute tp4
@ -198,7 +197,7 @@ public class RecordAccumulatorTest {
accum.append(topic, partition3, 0L, key, value, Record.EMPTY_HEADERS, null, maxBlockTimeMs, time.milliseconds(), cluster);
accum.unmutePartition(tp4);
// set maxSize as a max value, so that the all partitions in 2 nodes should be drained: node1 => [tp1, tp2], node2 => [tp3, tp4]
Map<Integer, List<ProducerBatch>> batches5 = accum.drain(metadataCache, new HashSet<>(Arrays.asList(node1, node2)), Integer.MAX_VALUE, 0);
Map<Integer, List<ProducerBatch>> batches5 = accum.drain(metadataCache, Set.of(node1, node2), Integer.MAX_VALUE, 0);
verifyTopicPartitionInBatches(batches5, tp1, tp2, tp3, tp4);
}
@ -1430,7 +1429,7 @@ public class RecordAccumulatorTest {
// Try to drain from node1, it should return no batches.
Map<Integer, List<ProducerBatch>> batches = accum.drain(metadataCache,
new HashSet<>(Collections.singletonList(node1)), 999999 /* maxSize */, now);
Set.of(node1), 999999 /* maxSize */, now);
assertTrue(batches.containsKey(node1.id()) && batches.get(node1.id()).isEmpty(),
"No batches ready to be drained on Node1");
}
@ -1511,7 +1510,7 @@ public class RecordAccumulatorTest {
// Drain for node2, it should return 0 batches,
Map<Integer, List<ProducerBatch>> batches = accum.drain(metadataCache,
new HashSet<>(Collections.singletonList(node2)), 999999 /* maxSize */, time.milliseconds());
Set.of(node2), 999999 /* maxSize */, time.milliseconds());
assertTrue(batches.get(node2.id()).isEmpty());
}

View File

@ -3313,8 +3313,8 @@ public class SenderTest {
int tp0LeaderEpoch = 100;
int epoch = tp0LeaderEpoch;
this.client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1))),
RequestTestUtils.metadataUpdateWithIds(1, Set.of(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1)),
tp -> {
if (tp0.equals(tp)) {
return epoch;
@ -3341,8 +3341,8 @@ public class SenderTest {
// Update leader epoch for tp0
int newEpoch = ++tp0LeaderEpoch;
this.client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1))),
RequestTestUtils.metadataUpdateWithIds(1, Set.of(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1)),
tp -> {
if (tp0.equals(tp)) {
return newEpoch;
@ -3429,8 +3429,8 @@ public class SenderTest {
int tp1LeaderEpoch = 200;
int tp2LeaderEpoch = 300;
this.client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2))),
RequestTestUtils.metadataUpdateWithIds(1, Set.of(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2)),
tp -> {
if (tp0.equals(tp)) {
return tp0LeaderEpoch;
@ -3509,8 +3509,8 @@ public class SenderTest {
int tp1LeaderEpoch = 200;
int tp2LeaderEpoch = 300;
this.client.updateMetadata(
RequestTestUtils.metadataUpdateWithIds(1, new HashSet<>(Arrays.asList(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2))),
RequestTestUtils.metadataUpdateWithIds(1, Set.of(new TopicIdPartition(TOPIC_ID, tp0),
new TopicIdPartition(TOPIC_ID, tp1), new TopicIdPartition(TOPIC_ID, tp2)),
tp -> {
if (tp0.equals(tp)) {
return tp0LeaderEpoch;

View File

@ -416,7 +416,7 @@ public class ConfigDefTest {
.define("a", Type.STRING, Importance.LOW, "docs")
.define("b", Type.STRING, Importance.LOW, "docs");
Set<String> names = configDef.names();
assertEquals(new HashSet<>(Arrays.asList("a", "b")), names);
assertEquals(Set.of("a", "b"), names);
// should be unmodifiable
try {
names.add("new");
@ -439,13 +439,13 @@ public class ConfigDefTest {
// Creating a ConfigDef based on another should compute the correct number of configs with no parent, even
// if the base ConfigDef has already computed its parentless configs
final ConfigDef baseConfigDef = new ConfigDef().define("a", Type.STRING, Importance.LOW, "docs");
assertEquals(new HashSet<>(singletonList("a")), baseConfigDef.getConfigsWithNoParent());
assertEquals(Set.of("a"), baseConfigDef.getConfigsWithNoParent());
final ConfigDef configDef = new ConfigDef(baseConfigDef)
.define("parent", Type.STRING, Importance.HIGH, "parent docs", "group", 1, Width.LONG, "Parent", singletonList("child"))
.define("child", Type.STRING, Importance.HIGH, "docs");
assertEquals(new HashSet<>(Arrays.asList("a", "parent")), configDef.getConfigsWithNoParent());
assertEquals(Set.of("a", "parent"), configDef.getConfigsWithNoParent());
}

View File

@ -22,10 +22,8 @@ import org.apache.kafka.common.config.ConfigException;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -88,7 +86,7 @@ class EnvVarConfigProviderTest {
@Test
void testGetEnvVarsByKeyList() {
Set<String> keyList = new HashSet<>(Arrays.asList("test_var1", "secret_var2"));
Set<String> keyList = Set.of("test_var1", "secret_var2");
Set<String> keys = envVarConfigProvider.get(null, keyList).data().keySet();
assertEquals(keyList, keys);
}

View File

@ -31,6 +31,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.security.auth.callback.Callback;
import javax.security.auth.callback.UnsupportedCallbackException;
@ -86,7 +87,7 @@ public class OAuthBearerUnsecuredLoginCallbackHandlerTest {
assertNotNull(jws, "create token failed");
long startMs = mockTime.milliseconds();
confirmCorrectValues(jws, user, startMs, 1000 * 60 * 60);
assertEquals(new HashSet<>(Arrays.asList("sub", "iat", "exp")), jws.claims().keySet());
assertEquals(Set.of("sub", "iat", "exp"), jws.claims().keySet());
}
@SuppressWarnings("unchecked")
@ -123,11 +124,11 @@ public class OAuthBearerUnsecuredLoginCallbackHandlerTest {
long startMs = mockTime.milliseconds();
confirmCorrectValues(jws, user, startMs, lifetimeSeconds * 1000);
Map<String, Object> claims = jws.claims();
assertEquals(new HashSet<>(Arrays.asList(actualScopeClaimName, principalClaimName, "iat", "exp", "number",
"list", "emptyList1", "emptyList2")), claims.keySet());
assertEquals(new HashSet<>(Arrays.asList(explicitScope1, explicitScope2)),
assertEquals(Set.of(actualScopeClaimName, principalClaimName, "iat", "exp", "number",
"list", "emptyList1", "emptyList2"), claims.keySet());
assertEquals(Set.of(explicitScope1, explicitScope2),
new HashSet<>((List<String>) claims.get(actualScopeClaimName)));
assertEquals(new HashSet<>(Arrays.asList(explicitScope1, explicitScope2)), jws.scope());
assertEquals(Set.of(explicitScope1, explicitScope2), jws.scope());
assertEquals(1.0, jws.claim("number", Number.class));
assertEquals(Arrays.asList("1", "2", ""), jws.claim("list", List.class));
assertEquals(Collections.emptyList(), jws.claim("emptyList1", List.class));

View File

@ -26,9 +26,7 @@ import org.slf4j.LoggerFactory;
import java.nio.charset.StandardCharsets;
import java.security.Principal;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Base64;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.function.Predicate;
@ -54,10 +52,10 @@ import jakarta.ws.rs.core.SecurityContext;
public class JaasBasicAuthFilter implements ContainerRequestFilter {
private static final Logger log = LoggerFactory.getLogger(JaasBasicAuthFilter.class);
private static final Set<RequestMatcher> INTERNAL_REQUEST_MATCHERS = new HashSet<>(Arrays.asList(
private static final Set<RequestMatcher> INTERNAL_REQUEST_MATCHERS = Set.of(
new RequestMatcher(HttpMethod.POST, "/?connectors/([^/]+)/tasks/?"),
new RequestMatcher(HttpMethod.PUT, "/?connectors/[^/]+/fence/?")
));
);
private static final String CONNECT_LOGIN_MODULE = "KafkaConnect";
static final String AUTHORIZATION = "Authorization";

View File

@ -626,8 +626,8 @@ public class JsonConverterTest {
Set<JsonNode> payloadEntries = new HashSet<>();
for (JsonNode elem : payload)
payloadEntries.add(elem);
assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add(1).add(12),
JsonNodeFactory.instance.arrayNode().add(2).add(15))),
assertEquals(Set.of(JsonNodeFactory.instance.arrayNode().add(1).add(12),
JsonNodeFactory.instance.arrayNode().add(2).add(15)),
payloadEntries
);
}
@ -805,9 +805,9 @@ public class JsonConverterTest {
Set<JsonNode> payloadEntries = new HashSet<>();
for (JsonNode elem : payload)
payloadEntries.add(elem);
assertEquals(new HashSet<>(Arrays.asList(JsonNodeFactory.instance.arrayNode().add("string").add(12),
assertEquals(Set.of(JsonNodeFactory.instance.arrayNode().add("string").add(12),
JsonNodeFactory.instance.arrayNode().add(52).add("string"),
JsonNodeFactory.instance.arrayNode().add(false).add(true))),
JsonNodeFactory.instance.arrayNode().add(false).add(true)),
payloadEntries
);
}

View File

@ -97,8 +97,8 @@ public class MirrorClientTest {
MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "heartbeats",
"source1.heartbeats", "source2.source1.heartbeats", "source3.heartbeats"));
Set<String> heartbeatTopics = client.heartbeatTopics();
assertEquals(heartbeatTopics, new HashSet<>(Arrays.asList("heartbeats", "source1.heartbeats",
"source2.source1.heartbeats", "source3.heartbeats")));
assertEquals(heartbeatTopics, Set.of("heartbeats", "source1.heartbeats",
"source2.source1.heartbeats", "source3.heartbeats"));
}
@Test
@ -106,8 +106,8 @@ public class MirrorClientTest {
MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "checkpoints.internal",
"source1.checkpoints.internal", "source2.source1.checkpoints.internal", "source3.checkpoints.internal"));
Set<String> checkpointTopics = client.checkpointTopics();
assertEquals(new HashSet<>(Arrays.asList("source1.checkpoints.internal",
"source2.source1.checkpoints.internal", "source3.checkpoints.internal")), checkpointTopics);
assertEquals(Set.of("source1.checkpoints.internal",
"source2.source1.checkpoints.internal", "source3.checkpoints.internal"), checkpointTopics);
}
@Test

View File

@ -26,9 +26,7 @@ import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.metrics.stats.Min;
import org.apache.kafka.common.metrics.stats.Value;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
@ -38,7 +36,7 @@ class MirrorCheckpointMetrics implements AutoCloseable {
private static final String CHECKPOINT_CONNECTOR_GROUP = MirrorCheckpointConnector.class.getSimpleName();
private static final Set<String> GROUP_TAGS = new HashSet<>(Arrays.asList("source", "target", "group", "topic", "partition"));
private static final Set<String> GROUP_TAGS = Set.of("source", "target", "group", "topic", "partition");
private static final MetricNameTemplate CHECKPOINT_LATENCY = new MetricNameTemplate(
"checkpoint-latency-ms", CHECKPOINT_CONNECTOR_GROUP,

View File

@ -27,8 +27,6 @@ import org.apache.kafka.common.metrics.stats.Meter;
import org.apache.kafka.common.metrics.stats.Min;
import org.apache.kafka.common.metrics.stats.Value;
import java.util.Arrays;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Set;
@ -62,7 +60,7 @@ class MirrorSourceMetrics implements AutoCloseable {
this.source = taskConfig.sourceClusterAlias();
this.metrics = new Metrics();
Set<String> partitionTags = new HashSet<>(Arrays.asList("source", "target", "topic", "partition"));
Set<String> partitionTags = Set.of("source", "target", "topic", "partition");
recordCount = new MetricNameTemplate(
"record-count", SOURCE_CONNECTOR_GROUP,

View File

@ -35,9 +35,7 @@ import org.apache.kafka.connect.storage.OffsetStorageReader;
import org.junit.jupiter.api.Test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@ -224,14 +222,14 @@ public class MirrorSourceTaskTest {
OffsetStorageReader mockOffsetStorageReader = mock(OffsetStorageReader.class);
when(mockSourceTaskContext.offsetStorageReader()).thenReturn(mockOffsetStorageReader);
Set<TopicPartition> topicPartitions = new HashSet<>(Arrays.asList(
Set<TopicPartition> topicPartitions = Set.of(
new TopicPartition("previouslyReplicatedTopic", 8),
new TopicPartition("previouslyReplicatedTopic1", 0),
new TopicPartition("previouslyReplicatedTopic", 1),
new TopicPartition("newTopicToReplicate1", 1),
new TopicPartition("newTopicToReplicate1", 4),
new TopicPartition("newTopicToReplicate2", 0)
));
);
long arbitraryCommittedOffset = 4L;
long offsetToSeek = arbitraryCommittedOffset + 1L;

View File

@ -1101,7 +1101,7 @@ public class MirrorConnectorsIntegrationBaseTest {
}
protected static void alterMirrorMakerSourceConnectorOffsets(EmbeddedConnectCluster connectCluster, LongUnaryOperator alterOffset, String... topics) {
Set<String> topicsSet = new HashSet<>(Arrays.asList(topics));
Set<String> topicsSet = Set.of(topics);
String connectorName = MirrorSourceConnector.class.getSimpleName();
ConnectorOffsets currentOffsets = connectCluster.connectorOffsets(connectorName);
@ -1131,7 +1131,7 @@ public class MirrorConnectorsIntegrationBaseTest {
}
protected static void resetSomeMirrorMakerSourceConnectorOffsets(EmbeddedConnectCluster connectCluster, String... topics) {
Set<String> topicsSet = new HashSet<>(Arrays.asList(topics));
Set<String> topicsSet = Set.of(topics);
String connectorName = MirrorSourceConnector.class.getSimpleName();
ConnectorOffsets currentOffsets = connectCluster.connectorOffsets(connectorName);

View File

@ -45,7 +45,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -271,7 +270,7 @@ public class Plugins {
public String pluginVersion(String classOrAlias, ClassLoader sourceLoader, PluginType... allowedTypes) {
String location = (sourceLoader instanceof PluginClassLoader) ? ((PluginClassLoader) sourceLoader).location() : null;
PluginDesc<?> desc = delegatingLoader.pluginDesc(classOrAlias, location, new HashSet<>(Arrays.asList(allowedTypes)));
PluginDesc<?> desc = delegatingLoader.pluginDesc(classOrAlias, location, Set.of(allowedTypes));
if (desc != null) {
return desc.version();
}

View File

@ -483,7 +483,7 @@ public class WorkerSinkTaskTest {
workerTask.initializeAndStart();
verifyInitializeTask();
Set<TopicPartition> newAssignment = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3));
Set<TopicPartition> newAssignment = Set.of(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
when(consumer.assignment())
.thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT)
@ -638,8 +638,8 @@ public class WorkerSinkTaskTest {
.thenReturn(INITIAL_ASSIGNMENT)
.thenReturn(Collections.singleton(TOPIC_PARTITION2))
.thenReturn(Collections.singleton(TOPIC_PARTITION2))
.thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3)))
.thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3)))
.thenReturn(Set.of(TOPIC_PARTITION2, TOPIC_PARTITION3))
.thenReturn(Set.of(TOPIC_PARTITION2, TOPIC_PARTITION3))
.thenReturn(INITIAL_ASSIGNMENT)
.thenReturn(INITIAL_ASSIGNMENT)
.thenReturn(INITIAL_ASSIGNMENT);
@ -710,12 +710,12 @@ public class WorkerSinkTaskTest {
when(consumer.assignment())
.thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT)
.thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2)))
.thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2)))
.thenReturn(new HashSet<>(Collections.singletonList(TOPIC_PARTITION2)))
.thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3)))
.thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3)))
.thenReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION2, TOPIC_PARTITION3)));
.thenReturn(Set.of(TOPIC_PARTITION2))
.thenReturn(Set.of(TOPIC_PARTITION2))
.thenReturn(Set.of(TOPIC_PARTITION2))
.thenReturn(Set.of(TOPIC_PARTITION2, TOPIC_PARTITION3))
.thenReturn(Set.of(TOPIC_PARTITION2, TOPIC_PARTITION3))
.thenReturn(Set.of(TOPIC_PARTITION2, TOPIC_PARTITION3));
INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET));
when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET);

View File

@ -65,7 +65,6 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@ -111,8 +110,8 @@ public class WorkerSinkTaskThreadedTest {
private static final TopicPartition TOPIC_PARTITION2 = new TopicPartition(TOPIC, PARTITION2);
private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3);
private static final TopicPartition UNASSIGNED_TOPIC_PARTITION = new TopicPartition(TOPIC, 200);
private static final Set<TopicPartition> INITIAL_ASSIGNMENT = new HashSet<>(Arrays.asList(
TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3));
private static final Set<TopicPartition> INITIAL_ASSIGNMENT = Set.of(
TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
private static final Map<String, String> TASK_PROPS = new HashMap<>();
private static final long TIMESTAMP = 42L;
@ -439,7 +438,7 @@ public class WorkerSinkTaskThreadedTest {
doAnswer(invocation -> {
return null; // initial assignment
}).doAnswer(invocation -> {
assertEquals(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3)), sinkTaskContext.getValue().assignment());
assertEquals(Set.of(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3), sinkTaskContext.getValue().assignment());
return null;
}).doAnswer(invocation -> {
try {

View File

@ -1031,7 +1031,7 @@ public class DistributedHerderTest {
// tasks are revoked
TopicStatus fooStatus = new TopicStatus(FOO_TOPIC, CONN1, 0, time.milliseconds());
TopicStatus barStatus = new TopicStatus(BAR_TOPIC, CONN1, 0, time.milliseconds());
when(statusBackingStore.getAllTopics(eq(CONN1))).thenReturn(new HashSet<>(Arrays.asList(fooStatus, barStatus)));
when(statusBackingStore.getAllTopics(eq(CONN1))).thenReturn(Set.of(fooStatus, barStatus));
doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(FOO_TOPIC));
doNothing().when(statusBackingStore).deleteTopic(eq(CONN1), eq(BAR_TOPIC));
@ -3232,7 +3232,7 @@ public class DistributedHerderTest {
taskConfigGenerations.put(CONN1, 3);
taskConfigGenerations.put(CONN2, 4);
taskConfigGenerations.put(conn3, 2);
Set<String> pendingFencing = new HashSet<>(Arrays.asList(CONN1, CONN2, conn3));
Set<String> pendingFencing = Set.of(CONN1, CONN2, conn3);
ClusterConfigState configState = exactlyOnceSnapshot(
sessionKey,
TASK_CONFIGS_MAP,

View File

@ -923,7 +923,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
Set<String> expectedWorkers = new HashSet<>(Arrays.asList(newWorker, flakyWorker));
Set<String> expectedWorkers = Set.of(newWorker, flakyWorker);
assertEquals(expectedWorkers,
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
@ -1455,7 +1455,7 @@ public class IncrementalCooperativeAssignorTest {
}
private void assertWorkers(String... workers) {
assertEquals(new HashSet<>(Arrays.asList(workers)), returnedAssignments.allWorkers(), "Wrong set of workers");
assertEquals(Set.of(workers), returnedAssignments.allWorkers(), "Wrong set of workers");
}
/**

View File

@ -200,7 +200,7 @@ public class ConnectorsResourceTest {
Collection<String> connectors = (Collection<String>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets
assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors));
assertEquals(Set.of(CONNECTOR_NAME, CONNECTOR2_NAME), new HashSet<>(connectors));
}
@Test
@ -218,7 +218,7 @@ public class ConnectorsResourceTest {
Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets
assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), expanded.keySet());
assertEquals(Set.of(CONNECTOR_NAME, CONNECTOR2_NAME), expanded.keySet());
assertEquals(connector2, expanded.get(CONNECTOR2_NAME).get("status"));
assertEquals(connector, expanded.get(CONNECTOR_NAME).get("status"));
}
@ -238,7 +238,7 @@ public class ConnectorsResourceTest {
Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets
assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), expanded.keySet());
assertEquals(Set.of(CONNECTOR_NAME, CONNECTOR2_NAME), expanded.keySet());
assertEquals(connector2, expanded.get(CONNECTOR2_NAME).get("info"));
assertEquals(connector, expanded.get(CONNECTOR_NAME).get("info"));
}
@ -262,7 +262,7 @@ public class ConnectorsResourceTest {
Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets
assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), expanded.keySet());
assertEquals(Set.of(CONNECTOR_NAME, CONNECTOR2_NAME), expanded.keySet());
assertEquals(connectorInfo2, expanded.get(CONNECTOR2_NAME).get("info"));
assertEquals(connectorInfo, expanded.get(CONNECTOR_NAME).get("info"));
assertEquals(connector2, expanded.get(CONNECTOR2_NAME).get("status"));

View File

@ -35,9 +35,9 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import static org.apache.kafka.connect.json.JsonConverterConfig.SCHEMAS_ENABLE_CONFIG;
@ -277,7 +277,7 @@ public class KafkaStatusBackingStoreFormatTest {
assertEquals(secondTopicStatus, store.parseTopicStatus(valueCaptor.getValue()));
assertEquals(firstTopicStatus, store.getTopic(FOO_CONNECTOR, FOO_TOPIC));
assertEquals(secondTopicStatus, store.getTopic(FOO_CONNECTOR, BAR_TOPIC));
assertEquals(new HashSet<>(Arrays.asList(firstTopicStatus, secondTopicStatus)), new HashSet<>(store.getAllTopics(FOO_CONNECTOR)));
assertEquals(Set.of(firstTopicStatus, secondTopicStatus), new HashSet<>(store.getAllTopics(FOO_CONNECTOR)));
}
}

View File

@ -43,11 +43,11 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Supplier;
import static org.apache.kafka.clients.CommonClientConfigs.CLIENT_ID_CONFIG;
@ -388,8 +388,8 @@ public class KafkaStatusBackingStoreTest {
verify(kafkaBasedLog).send(eq("status-connector-" + CONNECTOR), eq(value), any(Callback.class));
verify(kafkaBasedLog).send(eq("status-task-conn-0"), eq(value), any(Callback.class));
assertEquals(new HashSet<>(Collections.singletonList(CONNECTOR)), store.connectors());
assertEquals(new HashSet<>(Collections.singletonList(taskStatus)), new HashSet<>(store.getAll(CONNECTOR)));
assertEquals(Set.of(CONNECTOR), store.connectors());
assertEquals(Set.of(taskStatus), new HashSet<>(store.getAll(CONNECTOR)));
store.read(consumerRecord(0, "status-connector-conn", null));
assertTrue(store.connectors().isEmpty());
assertTrue(store.getAll(CONNECTOR).isEmpty());
@ -412,7 +412,7 @@ public class KafkaStatusBackingStoreTest {
verify(kafkaBasedLog).send(eq("status-task-conn-0"), eq(value), any(Callback.class));
assertEquals(new HashSet<>(Collections.singletonList(taskStatus)), new HashSet<>(store.getAll(CONNECTOR)));
assertEquals(Set.of(taskStatus), new HashSet<>(store.getAll(CONNECTOR)));
store.read(consumerRecord(0, "status-task-conn-0", null));
assertTrue(store.getAll(CONNECTOR).isEmpty());
}

View File

@ -51,7 +51,6 @@ import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
@ -100,7 +99,7 @@ public class KafkaBasedLogTest {
CONSUMER_PROPS.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
}
private static final Set<TopicPartition> CONSUMER_ASSIGNMENT = new HashSet<>(Arrays.asList(TP0, TP1));
private static final Set<TopicPartition> CONSUMER_ASSIGNMENT = Set.of(TP0, TP1);
private static final Map<String, String> FIRST_SET = new HashMap<>();
static {
FIRST_SET.put("key", "value");
@ -408,7 +407,7 @@ public class KafkaBasedLogTest {
@Test
public void testOffsetReadFailureWhenWorkThreadFails() throws Exception {
RuntimeException exception = new RuntimeException();
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Set<TopicPartition> tps = Set.of(TP0, TP1);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
@ -482,7 +481,7 @@ public class KafkaBasedLogTest {
@Test
public void testReadEndOffsetsUsingAdmin() {
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Set<TopicPartition> tps = Set.of(TP0, TP1);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);
@ -498,7 +497,7 @@ public class KafkaBasedLogTest {
@Test
public void testReadEndOffsetsUsingAdminThatFailsWithUnsupported() {
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Set<TopicPartition> tps = Set.of(TP0, TP1);
admin = mock(TopicAdmin.class);
// Getting end offsets using the admin client should fail with unsupported version
when(admin.retryEndOffsets(eq(tps), any(), anyLong())).thenThrow(new UnsupportedVersionException("too old"));
@ -516,7 +515,7 @@ public class KafkaBasedLogTest {
@Test
public void testReadEndOffsetsUsingAdminThatFailsWithRetriable() {
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(TP0, TP1));
Set<TopicPartition> tps = Set.of(TP0, TP1);
Map<TopicPartition, Long> endOffsets = new HashMap<>();
endOffsets.put(TP0, 0L);
endOffsets.put(TP1, 0L);

View File

@ -65,7 +65,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -667,7 +666,7 @@ public class TopicAdminTest {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
TopicPartition tp2 = new TopicPartition(topicName, 1);
Set<TopicPartition> tps = new HashSet<>(Arrays.asList(tp1, tp2));
Set<TopicPartition> tps = Set.of(tp1, tp2);
long offset1 = 1001;
long offset2 = 1002;
Cluster cluster = createCluster(1, topicName, 2);

View File

@ -33,12 +33,11 @@ import org.apache.kafka.connect.transforms.RegexRouter;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.common.config.TopicConfig.CLEANUP_POLICY_COMPACT;
@ -136,7 +135,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(topicCreation.defaultTopicGroup(), groups.get(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(2, topicCreation.topicGroups().size());
assertEquals(new HashSet<>(Arrays.asList(FOO_GROUP, BAR_GROUP)), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP, BAR_GROUP), topicCreation.topicGroups().keySet());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -300,7 +299,7 @@ public class TopicCreationTest {
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(2, groups.size());
assertEquals(new HashSet<>(Arrays.asList(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP)), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -365,7 +364,7 @@ public class TopicCreationTest {
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(2, groups.size());
assertEquals(new HashSet<>(Arrays.asList(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP)), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -442,7 +441,7 @@ public class TopicCreationTest {
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(3, groups.size());
assertEquals(new HashSet<>(Arrays.asList(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP, BAR_GROUP)), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP, BAR_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -466,7 +465,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC));
assertEquals(2, topicCreation.topicGroups().size());
assertEquals(new HashSet<>(Arrays.asList(FOO_GROUP, BAR_GROUP)), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP, BAR_GROUP), topicCreation.topicGroups().keySet());
assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC));
assertEquals(barGroup, topicCreation.findFirstGroup(BAR_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
@ -514,7 +513,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(groups.get(DEFAULT_TOPIC_CREATION_GROUP), topicCreation.defaultTopicGroup());
assertEquals(2, topicCreation.topicGroups().size());
assertEquals(new HashSet<>(Arrays.asList(FOO_GROUP, BAR_GROUP)), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP, BAR_GROUP), topicCreation.topicGroups().keySet());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -574,7 +573,7 @@ public class TopicCreationTest {
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(3, groups.size());
assertEquals(new HashSet<>(Arrays.asList(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP, BAR_GROUP)), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP, BAR_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -598,7 +597,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC));
assertEquals(2, topicCreation.topicGroups().size());
assertEquals(new HashSet<>(Arrays.asList(FOO_GROUP, BAR_GROUP)), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP, BAR_GROUP), topicCreation.topicGroups().keySet());
assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC));
assertEquals(barGroup, topicCreation.findFirstGroup(BAR_TOPIC));
topicCreation.addTopic(FOO_TOPIC);

View File

@ -25,7 +25,6 @@ import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
@ -147,7 +146,7 @@ public class ConnectAssertions {
* @param topicNames the names of the topics that are expected to not exist
*/
public void assertTopicsDoNotExist(String... topicNames) throws InterruptedException {
Set<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
Set<String> topicNameSet = Set.of(topicNames);
AtomicReference<Set<String>> existingTopics = new AtomicReference<>(topicNameSet);
waitForCondition(
() -> checkTopicsExist(topicNameSet, (actual, expected) -> {
@ -164,7 +163,7 @@ public class ConnectAssertions {
* @param topicNames the names of the topics that are expected to exist
*/
public void assertTopicsExist(String... topicNames) throws InterruptedException {
Set<String> topicNameSet = new HashSet<>(Arrays.asList(topicNames));
Set<String> topicNameSet = Set.of(topicNames);
AtomicReference<Set<String>> missingTopics = new AtomicReference<>(topicNameSet);
waitForCondition(
() -> checkTopicsExist(topicNameSet, (actual, expected) -> {

View File

@ -65,7 +65,6 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -282,7 +281,7 @@ public class EmbeddedKafkaCluster {
* @return the map of optional {@link TopicDescription} keyed by the topic name
*/
public Map<String, Optional<TopicDescription>> describeTopics(String... topicNames) {
return describeTopics(new HashSet<>(Arrays.asList(topicNames)));
return describeTopics(Set.of(topicNames));
}
/**

View File

@ -27,8 +27,7 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.IntStream;
import static org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetricsImpl.BATCH_FLUSH_TIME_METRIC_NAME;
@ -49,7 +48,7 @@ public class CoordinatorRuntimeMetricsImplTest {
public void testMetricNames() {
Metrics metrics = new Metrics();
HashSet<org.apache.kafka.common.MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
Set<org.apache.kafka.common.MetricName> expectedMetrics = Set.of(
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "loading"),
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "active"),
kafkaMetricName(metrics, NUM_PARTITIONS_METRIC_NAME, "state", "failed"),
@ -77,7 +76,7 @@ public class CoordinatorRuntimeMetricsImplTest {
kafkaMetricName(metrics, "batch-flush-time-ms-p95"),
kafkaMetricName(metrics, "batch-flush-time-ms-p99"),
kafkaMetricName(metrics, "batch-flush-time-ms-p999")
));
);
try (CoordinatorRuntimeMetricsImpl runtimeMetrics = new CoordinatorRuntimeMetricsImpl(metrics, METRICS_GROUP)) {
runtimeMetrics.registerEventQueueSizeGauge(() -> 0);

View File

@ -136,8 +136,8 @@ public class MiniKdc {
* MiniKdc.
*/
public MiniKdc(Properties config, File workDir) {
Set<String> requiredProperties = new HashSet<>(List.of(ORG_NAME, ORG_DOMAIN, KDC_BIND_ADDRESS, KDC_PORT,
INSTANCE, TRANSPORT, MAX_TICKET_LIFETIME, MAX_RENEWABLE_LIFETIME));
Set<String> requiredProperties = Set.of(ORG_NAME, ORG_DOMAIN, KDC_BIND_ADDRESS, KDC_PORT,
INSTANCE, TRANSPORT, MAX_TICKET_LIFETIME, MAX_RENEWABLE_LIFETIME);
if (!config.keySet().containsAll(requiredProperties)) {
throw new IllegalArgumentException("Missing required properties: " + requiredProperties);
}

View File

@ -29,8 +29,6 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.TreeMap;
@ -119,7 +117,7 @@ public class ReconfigurableQuorumIntegrationTest {
try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(new HashSet<>(List.of(3000, 3001, 3002)), voters.keySet());
assertEquals(Set.of(3000, 3001, 3002), voters.keySet());
for (int replicaId : new int[] {3000, 3001, 3002}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
@ -144,7 +142,7 @@ public class ReconfigurableQuorumIntegrationTest {
try (Admin admin = Admin.create(cluster.clientProperties())) {
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(new HashSet<>(List.of(3000, 3001, 3002, 3003)), voters.keySet());
assertEquals(Set.of(3000, 3001, 3002, 3003), voters.keySet());
for (int replicaId : new int[] {3000, 3001, 3002, 3003}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}
@ -153,7 +151,7 @@ public class ReconfigurableQuorumIntegrationTest {
admin.removeRaftVoter(3000, dirId).all().get();
TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> {
Map<Integer, Uuid> voters = findVoterDirs(admin);
assertEquals(new HashSet<>(List.of(3001, 3002, 3003)), voters.keySet());
assertEquals(Set.of(3001, 3002, 3003), voters.keySet());
for (int replicaId : new int[] {3001, 3002, 3003}) {
assertNotEquals(Uuid.ZERO_UUID, voters.get(replicaId));
}

View File

@ -1013,7 +1013,7 @@ public class SharePartitionManagerTest {
ShareFetchResponse resp1 = context1.updateAndGenerateResponseData(groupId, reqMetadata1.memberId(), respData1);
assertEquals(Errors.NONE, resp1.error());
assertEquals(new HashSet<>(List.of(tp0, tp1)),
assertEquals(Set.of(tp0, tp1),
new HashSet<>(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId1)));
// Create a new share session with an initial share fetch request.
@ -1047,7 +1047,7 @@ public class SharePartitionManagerTest {
ShareFetchResponse resp3 = context3.updateAndGenerateResponseData(groupId, reqMetadata1.memberId(), respData3);
assertEquals(Errors.NONE, resp3.error());
assertEquals(new HashSet<>(List.of(tp0, tp1, tp2)),
assertEquals(Set.of(tp0, tp1, tp2),
new HashSet<>(sharePartitionManager.cachedTopicIdPartitionsInShareSession(groupId, memberId1)));
// Continue the second session we created.

View File

@ -17,8 +17,6 @@
package org.apache.kafka.coordinator.group.classic;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Locale;
import java.util.Set;
@ -131,7 +129,7 @@ public enum ClassicGroupState {
}
private void addValidPreviousStates(ClassicGroupState... validPreviousStates) {
this.validPreviousStates = new HashSet<>(Arrays.asList(validPreviousStates));
this.validPreviousStates = Set.of(validPreviousStates);
}
/**

View File

@ -18,7 +18,6 @@ package org.apache.kafka.coordinator.group.assignor;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Set;
@ -124,7 +123,7 @@ public class RangeSetTest {
RangeSet rangeSet2 = new RangeSet(5, 10);
RangeSet rangeSet3 = new RangeSet(6, 10);
Set<Integer> set = Set.of(5, 6, 7, 8, 9);
HashSet<Integer> hashSet = new HashSet<>(Set.of(6, 7, 8, 9));
Set<Integer> hashSet = Set.of(6, 7, 8, 9);
assertEquals(rangeSet1, rangeSet2);
assertNotEquals(rangeSet1, rangeSet3);

View File

@ -420,7 +420,7 @@ public class ClassicGroupTest {
group.initNextGeneration();
Set<String> expectedTopics = new HashSet<>(Set.of("foo"));
Set<String> expectedTopics = Set.of("foo");
assertEquals(expectedTopics, group.subscribedTopics().get());
group.transitionTo(PREPARING_REBALANCE);
@ -1275,14 +1275,14 @@ public class ClassicGroupTest {
group.transitionTo(COMPLETING_REBALANCE);
assertTrue(group.isInStates(new HashSet<>(List.of("completingrebalance")), 0));
assertTrue(group.isInStates(Set.of("completingrebalance"), 0));
group.transitionTo(STABLE);
assertTrue(group.isInStates(Set.of("stable"), 0));
assertFalse(group.isInStates(Set.of("empty"), 0));
group.transitionTo(DEAD);
assertTrue(group.isInStates(new HashSet<>(List.of("dead", " ")), 0));
assertTrue(group.isInStates(Set.of("dead", " "), 0));
}
@Test

View File

@ -36,8 +36,6 @@ import com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.stream.IntStream;
@ -63,7 +61,7 @@ public class GroupCoordinatorMetricsTest {
MetricsRegistry registry = new MetricsRegistry();
Metrics metrics = new Metrics();
HashSet<org.apache.kafka.common.MetricName> expectedMetrics = new HashSet<>(Arrays.asList(
Set<org.apache.kafka.common.MetricName> expectedMetrics = Set.of(
metrics.metricName("offset-commit-rate", GroupCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("offset-commit-count", GroupCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("offset-expiration-rate", GroupCoordinatorMetrics.METRICS_GROUP),
@ -159,11 +157,11 @@ public class GroupCoordinatorMetricsTest {
"streams-group-count",
GroupCoordinatorMetrics.METRICS_GROUP,
Map.of("state", StreamsGroupState.NOT_READY.toString()))
));
);
try {
try (GroupCoordinatorMetrics ignored = new GroupCoordinatorMetrics(registry, metrics)) {
HashSet<String> expectedRegistry = new HashSet<>(Arrays.asList(
Set<String> expectedRegistry = Set.of(
"kafka.coordinator.group:type=GroupMetadataManager,name=NumOffsets",
"kafka.coordinator.group:type=GroupMetadataManager,name=NumGroups",
"kafka.coordinator.group:type=GroupMetadataManager,name=NumGroupsPreparingRebalance",
@ -171,7 +169,7 @@ public class GroupCoordinatorMetricsTest {
"kafka.coordinator.group:type=GroupMetadataManager,name=NumGroupsStable",
"kafka.coordinator.group:type=GroupMetadataManager,name=NumGroupsDead",
"kafka.coordinator.group:type=GroupMetadataManager,name=NumGroupsEmpty"
));
);
assertMetricsForTypeEqual(registry, "kafka.coordinator.group", expectedRegistry);
expectedMetrics.forEach(metricName -> assertTrue(metrics.metrics().containsKey(metricName), metricName + " is missing"));

View File

@ -708,7 +708,7 @@ public class ConsumerGroupTest {
// Initial assignment for member1
Assignment initialAssignment = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(0))
Set.of(0)
));
consumerGroup.updateTargetAssignment(memberId1, initialAssignment);
@ -723,7 +723,7 @@ public class ConsumerGroupTest {
// New assignment for member1
Assignment newAssignment = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(1))
Set.of(1)
));
consumerGroup.updateTargetAssignment(memberId1, newAssignment);
@ -738,7 +738,7 @@ public class ConsumerGroupTest {
// New assignment for member2 to add partition 1
Assignment newAssignment2 = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(1))
Set.of(1)
));
consumerGroup.updateTargetAssignment(memberId2, newAssignment2);
@ -753,7 +753,7 @@ public class ConsumerGroupTest {
// New assignment for member1 to revoke partition 1 and assign partition 0
Assignment newAssignment1 = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(0))
Set.of(0)
));
consumerGroup.updateTargetAssignment(memberId1, newAssignment1);
@ -1119,8 +1119,8 @@ public class ConsumerGroupTest {
assertEquals(2, consumerGroup.classicMembersSupportedProtocols().get("range"));
assertEquals(1, consumerGroup.classicMembersSupportedProtocols().get("roundrobin"));
assertTrue(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, new HashSet<>(Arrays.asList("range", "sticky"))));
assertFalse(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, new HashSet<>(Arrays.asList("sticky", "roundrobin"))));
assertTrue(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, Set.of("range", "sticky")));
assertFalse(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, Set.of("sticky", "roundrobin")));
member2 = new ConsumerGroupMember.Builder(member2)
.setClassicMemberMetadata(new ConsumerGroupMemberMetadataValue.ClassicMemberMetadata()
@ -1144,7 +1144,7 @@ public class ConsumerGroupTest {
assertEquals(2, consumerGroup.classicMembersSupportedProtocols().get("range"));
assertEquals(2, consumerGroup.classicMembersSupportedProtocols().get("roundrobin"));
assertTrue(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, new HashSet<>(Arrays.asList("sticky", "roundrobin"))));
assertTrue(consumerGroup.supportsClassicProtocols(ConsumerProtocol.PROTOCOL_TYPE, Set.of("sticky", "roundrobin")));
}
@Test

View File

@ -36,7 +36,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -220,7 +219,7 @@ public class ShareGroupTest {
// Initial assignment for member1
Assignment initialAssignment = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(0))
Set.of(0)
));
shareGroup.updateTargetAssignment(memberId1, initialAssignment);
@ -235,7 +234,7 @@ public class ShareGroupTest {
// New assignment for member1
Assignment newAssignment = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(1))
Set.of(1)
));
shareGroup.updateTargetAssignment(memberId1, newAssignment);
@ -250,7 +249,7 @@ public class ShareGroupTest {
// New assignment for member2 to add partition 1
Assignment newAssignment2 = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(1))
Set.of(1)
));
shareGroup.updateTargetAssignment(memberId2, newAssignment2);
@ -265,7 +264,7 @@ public class ShareGroupTest {
// New assignment for member1 to revoke partition 1 and assign partition 0
Assignment newAssignment1 = new Assignment(Map.of(
topicId,
new HashSet<>(List.of(0))
Set.of(0)
));
shareGroup.updateTargetAssignment(memberId1, newAssignment1);

View File

@ -57,7 +57,7 @@ public class MockAssignorTest {
final AssignmentMemberSpec memberSpec1 = new AssignmentMemberSpec(
Optional.empty(),
Optional.empty(),
Map.of("test-subtopology", new HashSet<>(List.of(0))),
Map.of("test-subtopology", Set.of(0)),
Map.of(),
Map.of(),
"test-process",
@ -69,7 +69,7 @@ public class MockAssignorTest {
final AssignmentMemberSpec memberSpec2 = new AssignmentMemberSpec(
Optional.empty(),
Optional.empty(),
Map.of("test-subtopology", new HashSet<>(List.of(0))),
Map.of("test-subtopology", Set.of(0)),
Map.of(),
Map.of(),
"test-process",

View File

@ -20,7 +20,6 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -100,8 +99,8 @@ public class ConfiguredSubtopologyTest {
@Test
public void testAsStreamsGroupDescribeSubtopology() {
String subtopologyId = "subtopology1";
Set<String> sourceTopics = new HashSet<>(Set.of("sourceTopic1", "sourceTopic2"));
Set<String> repartitionSinkTopics = new HashSet<>(Set.of("repartitionSinkTopic1", "repartitionSinkTopic2"));
Set<String> sourceTopics = Set.of("sourceTopic1", "sourceTopic2");
Set<String> repartitionSinkTopics = Set.of("repartitionSinkTopic1", "repartitionSinkTopic2");
ConfiguredInternalTopic internalTopicMock = mock(ConfiguredInternalTopic.class);
StreamsGroupDescribeResponseData.TopicInfo topicInfo = new StreamsGroupDescribeResponseData.TopicInfo();
when(internalTopicMock.asStreamsGroupDescribeTopicInfo()).thenReturn(topicInfo);

View File

@ -324,9 +324,9 @@ public class AclControlManagerTest {
assertEquals(Optional.empty(), result.exception());
deleted.add(result.aclBinding());
}
assertEquals(new HashSet<>(List.of(
assertEquals(Set.of(
TEST_ACLS.get(0).toBinding(),
TEST_ACLS.get(2).toBinding())), deleted);
TEST_ACLS.get(2).toBinding()), deleted);
assertEquals(InvalidRequestException.class,
deleteResult.response().get(1).exception().get().getClass());
RecordTestUtils.replayAll(manager, deleteResult.records());

View File

@ -25,7 +25,6 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -37,7 +36,7 @@ public class BrokerToElrsTest {
};
private static Set<TopicIdPartition> toSet(TopicIdPartition... partitions) {
return new HashSet<>(List.of(partitions));
return Set.of(partitions);
}
private static Set<TopicIdPartition> toSet(BrokersToIsrs.PartitionsOnReplicaIterator iterator) {

View File

@ -27,7 +27,6 @@ import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -41,7 +40,7 @@ public class BrokersToIsrsTest {
};
private static Set<TopicIdPartition> toSet(TopicIdPartition... partitions) {
return new HashSet<>(List.of(partitions));
return Set.of(partitions);
}
private static Set<TopicIdPartition> toSet(PartitionsOnReplicaIterator iterator) {

View File

@ -41,11 +41,11 @@ import org.junit.jupiter.api.Timeout;
import java.util.AbstractMap.SimpleImmutableEntry;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -257,9 +257,8 @@ public class FeatureControlManagerTest {
assertEquals(MetadataVersion.MINIMUM_VERSION, manager.metadataVersionOrThrow());
assertEquals(Optional.of((short) 1), manager.finalizedFeatures(Long.MAX_VALUE).get(TestFeatureVersion.FEATURE_NAME));
assertEquals(Optional.of((short) 2), manager.finalizedFeatures(Long.MAX_VALUE).get(TransactionVersion.FEATURE_NAME));
assertEquals(new HashSet<>(List.of(
MetadataVersion.FEATURE_NAME, TestFeatureVersion.FEATURE_NAME, TransactionVersion.FEATURE_NAME)),
manager.finalizedFeatures(Long.MAX_VALUE).featureNames());
assertEquals(Set.of(MetadataVersion.FEATURE_NAME, TestFeatureVersion.FEATURE_NAME, TransactionVersion.FEATURE_NAME),
manager.finalizedFeatures(Long.MAX_VALUE).featureNames());
}
private FeatureControlManager createTestManager() {

View File

@ -660,7 +660,7 @@ public class QuorumControllerTest {
).iterator()));
CreateTopicsResponseData createTopicsResponseData = active.createTopics(
ANONYMOUS_CONTEXT, createTopicsRequestData,
new HashSet<>(List.of("foo", "bar"))).get();
Set.of("foo", "bar")).get();
assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("foo").errorCode()));
assertEquals(Errors.NONE, Errors.forCode(createTopicsResponseData.topics().find("bar").errorCode()));
Uuid topicIdFoo = createTopicsResponseData.topics().find("foo").topicId();

View File

@ -1566,7 +1566,7 @@ public class ReplicationControlManagerTest {
ctx.unfenceBrokers(0, 1, 3);
ControllerRequestContext requestContext = anonymousContextFor(ApiKeys.CREATE_TOPICS);
ControllerResult<CreateTopicsResponseData> createTopicResult = replicationControl.
createTopics(requestContext, request, new HashSet<>(List.of("foo", "bar", "quux", "foo2")));
createTopics(requestContext, request, Set.of("foo", "bar", "quux", "foo2"));
ctx.replay(createTopicResult.records());
List<CreatePartitionsTopic> topics = new ArrayList<>();
topics.add(new CreatePartitionsTopic().
@ -1690,7 +1690,7 @@ public class ReplicationControlManagerTest {
ControllerRequestContext requestContext =
anonymousContextFor(ApiKeys.CREATE_TOPICS);
ControllerResult<CreateTopicsResponseData> createTopicResult = replicationControl.
createTopics(requestContext, request, new HashSet<>(List.of("foo")));
createTopics(requestContext, request, Set.of("foo"));
ctx.replay(createTopicResult.records());
ctx.registerBrokers(0, 1);
@ -2961,12 +2961,12 @@ public class ReplicationControlManagerTest {
KRaftClusterDescriber describer = replication.clusterDescriber;
HashSet<UsableBroker> brokers = new HashSet<>();
describer.usableBrokers().forEachRemaining(broker -> brokers.add(broker));
assertEquals(new HashSet<>(List.of(
assertEquals(Set.of(
new UsableBroker(0, Optional.empty(), true),
new UsableBroker(1, Optional.empty(), true),
new UsableBroker(2, Optional.empty(), false),
new UsableBroker(3, Optional.empty(), false),
new UsableBroker(4, Optional.empty(), false))), brokers);
new UsableBroker(4, Optional.empty(), false)), brokers);
assertEquals(DirectoryId.MIGRATING, describer.defaultDir(1));
assertEquals(Uuid.fromString("ozwqsVMFSNiYQUPSJA3j0w"), describer.defaultDir(2));
assertEquals(DirectoryId.UNASSIGNED, describer.defaultDir(3));

View File

@ -26,8 +26,6 @@ import com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.BiConsumer;
@ -53,7 +51,7 @@ public class ControllerMetadataMetricsTest {
brokerRegistration(false, false)
);
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller:",
new HashSet<>(List.of(
Set.of(
"kafka.controller:type=KafkaController,name=ActiveBrokerCount",
"kafka.controller:type=KafkaController,name=FencedBrokerCount",
"kafka.controller:type=KafkaController,name=ControlledShutdownBrokerCount",
@ -67,7 +65,7 @@ public class ControllerMetadataMetricsTest {
"kafka.controller:type=KafkaController,name=IgnoredStaticVoters",
"kafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec",
"kafka.controller:type=ControllerStats,name=ElectionFromEligibleLeaderReplicasPerSec"
)));
));
}
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "KafkaController",
Set.of());

View File

@ -26,8 +26,6 @@ import com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
@ -44,7 +42,7 @@ public class QuorumControllerMetricsTest {
time,
9000)) {
metrics.addTimeSinceLastHeartbeatMetric(1);
HashSet<String> expected = new HashSet<>(List.of(
Set<String> expected = Set.of(
"kafka.controller:type=ControllerEventManager,name=EventQueueProcessingTimeMs",
"kafka.controller:type=ControllerEventManager,name=EventQueueTimeMs",
"kafka.controller:type=KafkaController,name=ActiveControllerCount",
@ -57,7 +55,7 @@ public class QuorumControllerMetricsTest {
"kafka.controller:type=KafkaController,name=NewActiveControllersCount",
"kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount",
"kafka.controller:type=KafkaController,name=TimeSinceLastHeartbeatReceivedMs,broker=1"
));
);
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller", expected);
}
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.controller",

View File

@ -250,11 +250,11 @@ public class TopicsImageTest {
LocalReplicaChanges changes = delta.localChanges(localId);
assertEquals(
new HashSet<>(List.of(new TopicPartition("baz", 0))),
Set.of(new TopicPartition("baz", 0)),
changes.electedLeaders().keySet()
);
assertEquals(
new HashSet<>(List.of(new TopicPartition("baz", 0))),
Set.of(new TopicPartition("baz", 0)),
changes.leaders().keySet()
);
assertEquals(
@ -306,7 +306,7 @@ public class TopicsImageTest {
RecordTestUtils.replayAll(delta, topicRecords);
LocalReplicaChanges changes = delta.localChanges(localId);
assertEquals(new HashSet<>(List.of(new TopicPartition("zoo", 0))), changes.deletes());
assertEquals(Set.of(new TopicPartition("zoo", 0)), changes.deletes());
assertEquals(Map.of(), changes.electedLeaders());
assertEquals(Map.of(), changes.leaders());
assertEquals(Map.of(), changes.followers());
@ -348,7 +348,7 @@ public class TopicsImageTest {
assertEquals(Set.of(), changes.deletes());
assertEquals(Map.of(), changes.electedLeaders());
assertEquals(
new HashSet<>(List.of(new TopicPartition("zoo", 0))),
Set.of(new TopicPartition("zoo", 0)),
changes.leaders().keySet()
);
assertEquals(Map.of(), changes.followers());
@ -567,19 +567,19 @@ public class TopicsImageTest {
LocalReplicaChanges changes = delta.localChanges(localId);
assertEquals(
new HashSet<>(List.of(new TopicPartition("zoo", 2), new TopicPartition("zoo", 3))),
Set.of(new TopicPartition("zoo", 2), new TopicPartition("zoo", 3)),
changes.deletes()
);
assertEquals(
new HashSet<>(List.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))),
Set.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4)),
changes.electedLeaders().keySet()
);
assertEquals(
new HashSet<>(List.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4))),
Set.of(new TopicPartition("zoo", 0), new TopicPartition("zoo", 4)),
changes.leaders().keySet()
);
assertEquals(
new HashSet<>(List.of(new TopicPartition("zoo", 1), new TopicPartition("zoo", 5))),
Set.of(new TopicPartition("zoo", 1), new TopicPartition("zoo", 5)),
changes.followers().keySet()
);
@ -647,9 +647,9 @@ public class TopicsImageTest {
assertNull(map.get("baz"));
HashSet<Uuid> uuids = new HashSet<>();
map.values().iterator().forEachRemaining(uuids::add);
HashSet<Uuid> expectedUuids = new HashSet<>(List.of(
Set<Uuid> expectedUuids = Set.of(
Uuid.fromString("ThIaNwRnSM2Nt9Mx1v0RvA"),
Uuid.fromString("f62ptyETTjet8SL5ZeREiw")));
Uuid.fromString("f62ptyETTjet8SL5ZeREiw"));
assertEquals(expectedUuids, uuids);
assertThrows(UnsupportedOperationException.class, () -> map.remove("foo"));
assertThrows(UnsupportedOperationException.class, () -> map.put("bar", FOO_UUID));
@ -666,7 +666,7 @@ public class TopicsImageTest {
assertNull(map.get(BAZ_UUID));
HashSet<String> names = new HashSet<>();
map.values().iterator().forEachRemaining(names::add);
HashSet<String> expectedNames = new HashSet<>(List.of("foo", "bar"));
Set<String> expectedNames = Set.of("foo", "bar");
assertEquals(expectedNames, names);
assertThrows(UnsupportedOperationException.class, () -> map.remove(FOO_UUID));
assertThrows(UnsupportedOperationException.class, () -> map.put(FOO_UUID, "bar"));

View File

@ -26,8 +26,6 @@ import com.yammer.metrics.core.MetricsRegistry;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
@ -70,11 +68,11 @@ public class MetadataLoaderMetricsTest {
try {
try (FakeMetadataLoaderMetrics fakeMetrics = new FakeMetadataLoaderMetrics(registry)) {
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.server",
new HashSet<>(List.of(
Set.of(
"kafka.server:type=MetadataLoader,name=CurrentControllerId",
"kafka.server:type=MetadataLoader,name=CurrentMetadataVersion",
"kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount"
)));
));
}
ControllerMetricsTestUtils.assertMetricsForTypeEqual(registry, "kafka.server",
Set.of());

View File

@ -32,10 +32,9 @@ import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.OptionalInt;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -91,7 +90,7 @@ public class ControllerRegistrationsPublisherTest {
build());
}
System.out.println("TEST_IMAGE.cluster = " + TEST_IMAGE.cluster());
assertEquals(new HashSet<>(List.of(0, 1, 2)), publisher.controllers().keySet());
assertEquals(Set.of(0, 1, 2), publisher.controllers().keySet());
assertTrue(publisher.controllers().get(0).zkMigrationReady());
assertFalse(publisher.controllers().get(1).zkMigrationReady());
assertFalse(publisher.controllers().get(2).zkMigrationReady());

View File

@ -29,8 +29,6 @@ import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashSet;
import java.util.List;
import java.util.Optional;
import java.util.Set;
@ -78,10 +76,10 @@ public class SnapshotEmitterMetricsTest {
public void testMetricNames() {
try (SnapshotEmitterMetricsTestContext ctx = new SnapshotEmitterMetricsTestContext()) {
ControllerMetricsTestUtils.assertMetricsForTypeEqual(ctx.registry, "kafka.server:",
new HashSet<>(List.of(
Set.of(
"kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes",
"kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs"
)));
));
ctx.metrics.close();
ControllerMetricsTestUtils.assertMetricsForTypeEqual(ctx.registry, "KafkaController",
Set.of());

View File

@ -20,7 +20,6 @@ package org.apache.kafka.metadata;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -109,9 +108,9 @@ public class ReplicasTest {
@Test
public void testToSet() {
assertEquals(Set.of(), Replicas.toSet(new int[] {}));
assertEquals(new HashSet<>(List.of(3, 1, 5)),
assertEquals(Set.of(3, 1, 5),
Replicas.toSet(new int[] {1, 3, 5}));
assertEquals(new HashSet<>(List.of(1, 2, 10)),
assertEquals(Set.of(1, 2, 10),
Replicas.toSet(new int[] {1, 1, 2, 10, 10}));
}

View File

@ -28,7 +28,6 @@ import java.nio.file.Files;
import java.util.AbstractMap.SimpleImmutableEntry;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -53,8 +52,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
public final class MetaPropertiesEnsembleTest {
private static final MetaPropertiesEnsemble FOO =
new MetaPropertiesEnsemble(
new HashSet<>(List.of("/tmp/empty1", "/tmp/empty2")),
new HashSet<>(List.of("/tmp/error3")),
Set.of("/tmp/empty1", "/tmp/empty2"),
Set.of("/tmp/error3"),
Stream.of(
new SimpleImmutableEntry<>("/tmp/dir4",
new MetaProperties.Builder().
@ -93,34 +92,34 @@ public final class MetaPropertiesEnsembleTest {
@Test
public void testEmptyLogDirsForFoo() {
assertEquals(new HashSet<>(List.of("/tmp/empty1", "/tmp/empty2")),
assertEquals(Set.of("/tmp/empty1", "/tmp/empty2"),
FOO.emptyLogDirs());
}
@Test
public void testEmptyLogDirsForEmpty() {
assertEquals(new HashSet<>(), EMPTY.emptyLogDirs());
assertEquals(Set.of(), EMPTY.emptyLogDirs());
}
@Test
public void testErrorLogDirsForFoo() {
assertEquals(new HashSet<>(List.of("/tmp/error3")), FOO.errorLogDirs());
assertEquals(Set.of("/tmp/error3"), FOO.errorLogDirs());
}
@Test
public void testErrorLogDirsForEmpty() {
assertEquals(new HashSet<>(), EMPTY.errorLogDirs());
assertEquals(Set.of(), EMPTY.errorLogDirs());
}
@Test
public void testLogDirPropsForFoo() {
assertEquals(new HashSet<>(List.of("/tmp/dir4", "/tmp/dir5")),
assertEquals(Set.of("/tmp/dir4", "/tmp/dir5"),
FOO.logDirProps().keySet());
}
@Test
public void testLogDirPropsForEmpty() {
assertEquals(new HashSet<>(),
assertEquals(Set.of(),
EMPTY.logDirProps().keySet());
}

View File

@ -56,13 +56,13 @@ import org.mockito.Mockito;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.OptionalInt;
import java.util.OptionalLong;
import java.util.Set;
import java.util.concurrent.ThreadLocalRandom;
import java.util.stream.IntStream;
import java.util.stream.Stream;
@ -1127,7 +1127,7 @@ public class KafkaRaftClientReconfigTest {
context.pollUntilRequest();
context.collectEndQuorumRequests(
epoch,
new HashSet<>(List.of(follower1.id(), follower2.id())),
Set.of(follower1.id(), follower2.id()),
Optional.empty()
);

View File

@ -29,8 +29,6 @@ import org.junit.jupiter.params.provider.ValueSource;
import java.net.InetSocketAddress;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
@ -85,7 +83,7 @@ public final class VoterSetTest {
@Test
void testVoterIds() {
VoterSet voterSet = VoterSet.fromMap(voterMap(IntStream.of(1, 2, 3), true));
assertEquals(new HashSet<>(List.of(1, 2, 3)), voterSet.voterIds());
assertEquals(Set.of(1, 2, 3), voterSet.voterIds());
}
@Test

View File

@ -24,10 +24,10 @@ import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.junit.jupiter.api.Test;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
@ -98,7 +98,7 @@ public final class EndpointReadyFuturesTest {
public void testImmediateCompletion() {
EndpointReadyFutures readyFutures = new EndpointReadyFutures.Builder().
build(Optional.empty(), INFO);
assertEquals(new HashSet<>(List.of(EXTERNAL, INTERNAL)),
assertEquals(Set.of(EXTERNAL, INTERNAL),
readyFutures.futures().keySet());
assertComplete(readyFutures, EXTERNAL, INTERNAL);
}
@ -109,7 +109,7 @@ public final class EndpointReadyFuturesTest {
EndpointReadyFutures readyFutures = new EndpointReadyFutures.Builder().
addReadinessFuture("foo", foo).
build(Optional.empty(), INFO);
assertEquals(new HashSet<>(List.of(EXTERNAL, INTERNAL)),
assertEquals(Set.of(EXTERNAL, INTERNAL),
readyFutures.futures().keySet());
assertIncomplete(readyFutures, EXTERNAL, INTERNAL);
foo.complete(null);
@ -124,7 +124,7 @@ public final class EndpointReadyFuturesTest {
addReadinessFuture("foo", foo).
addReadinessFuture("bar", bar).
build(Optional.empty(), INFO);
assertEquals(new HashSet<>(List.of(EXTERNAL, INTERNAL)),
assertEquals(Set.of(EXTERNAL, INTERNAL),
readyFutures.futures().keySet());
assertIncomplete(readyFutures, EXTERNAL, INTERNAL);
foo.complete(null);
@ -141,7 +141,7 @@ public final class EndpointReadyFuturesTest {
EndpointReadyFutures readyFutures = new EndpointReadyFutures.Builder().
addReadinessFutures("baz", bazFutures).
build(Optional.empty(), INFO);
assertEquals(new HashSet<>(List.of(EXTERNAL, INTERNAL)),
assertEquals(Set.of(EXTERNAL, INTERNAL),
readyFutures.futures().keySet());
assertIncomplete(readyFutures, EXTERNAL, INTERNAL);
bazFutures.get(EXTERNAL).complete(null);
@ -159,7 +159,7 @@ public final class EndpointReadyFuturesTest {
addReadinessFuture("foo", foo).
addReadinessFuture("bar", bar).
build(Optional.empty(), INFO);
assertEquals(new HashSet<>(List.of(EXTERNAL, INTERNAL)),
assertEquals(Set.of(EXTERNAL, INTERNAL),
readyFutures.futures().keySet());
assertIncomplete(readyFutures, EXTERNAL, INTERNAL);
foo.complete(null);

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.resource.ResourceType;
import org.apache.kafka.common.security.auth.KafkaPrincipal;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.stream.Collectors;
@ -54,17 +53,17 @@ public class AclEntry {
public static Set<AclOperation> supportedOperations(ResourceType resourceType) {
switch (resourceType) {
case TOPIC:
return new HashSet<>(Arrays.asList(READ, WRITE, CREATE, DESCRIBE, DELETE, ALTER, DESCRIBE_CONFIGS, ALTER_CONFIGS));
return Set.of(READ, WRITE, CREATE, DESCRIBE, DELETE, ALTER, DESCRIBE_CONFIGS, ALTER_CONFIGS);
case GROUP:
return new HashSet<>(Arrays.asList(READ, DESCRIBE, DELETE, DESCRIBE_CONFIGS, ALTER_CONFIGS));
return Set.of(READ, DESCRIBE, DELETE, DESCRIBE_CONFIGS, ALTER_CONFIGS);
case CLUSTER:
return new HashSet<>(Arrays.asList(CREATE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, ALTER, DESCRIBE));
return Set.of(CREATE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, ALTER, DESCRIBE);
case TRANSACTIONAL_ID:
return new HashSet<>(Arrays.asList(DESCRIBE, WRITE, TWO_PHASE_COMMIT));
return Set.of(DESCRIBE, WRITE, TWO_PHASE_COMMIT);
case DELEGATION_TOKEN:
return Set.of(DESCRIBE);
case USER:
return new HashSet<>(Arrays.asList(CREATE_TOKENS, DESCRIBE_TOKENS));
return Set.of(CREATE_TOKENS, DESCRIBE_TOKENS);
default:
throw new IllegalArgumentException("Not a concrete resource type");
}

View File

@ -212,7 +212,7 @@ class ShareCoordinatorServiceTest {
HashSet<WriteShareGroupStateResponseData.WriteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = new HashSet<>(List.of(
Set<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = Set.of(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new WriteShareGroupStateResponseData.PartitionResult()
@ -220,15 +220,15 @@ class ShareCoordinatorServiceTest {
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId1)
.setPartitions(List.of(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)))));
.setPartition(partition1))));
assertEquals(expectedResult, result);
verify(time, times(2)).hiResClockMs();
Set<MetricName> expectedMetrics = new HashSet<>(List.of(
Set<MetricName> expectedMetrics = Set.of(
metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP)
));
);
expectedMetrics.forEach(metric -> assertTrue(metrics.metrics().containsKey(metric)));
}
@ -329,9 +329,9 @@ class ShareCoordinatorServiceTest {
HashSet<ReadShareGroupStateResponseData.ReadStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = new HashSet<>(List.of(
Set<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = Set.of(
topicData1,
topicData2));
topicData2);
assertEquals(expectedResult, result);
}
@ -411,9 +411,9 @@ class ShareCoordinatorServiceTest {
HashSet<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> expectedResult = new HashSet<>(List.of(
Set<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> expectedResult = Set.of(
topicData1,
topicData2));
topicData2);
assertEquals(expectedResult, result);
}
@ -488,7 +488,7 @@ class ShareCoordinatorServiceTest {
HashSet<DeleteShareGroupStateResponseData.DeleteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<DeleteShareGroupStateResponseData.DeleteStateResult> expectedResult = new HashSet<>(List.of(
Set<DeleteShareGroupStateResponseData.DeleteStateResult> expectedResult = Set.of(
new DeleteShareGroupStateResponseData.DeleteStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult()
@ -496,7 +496,7 @@ class ShareCoordinatorServiceTest {
new DeleteShareGroupStateResponseData.DeleteStateResult()
.setTopicId(topicId1)
.setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)))));
.setPartition(partition1))));
assertEquals(expectedResult, result);
}
@ -573,7 +573,7 @@ class ShareCoordinatorServiceTest {
HashSet<InitializeShareGroupStateResponseData.InitializeStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<InitializeShareGroupStateResponseData.InitializeStateResult> expectedResult = new HashSet<>(List.of(
Set<InitializeShareGroupStateResponseData.InitializeStateResult> expectedResult = Set.of(
new InitializeShareGroupStateResponseData.InitializeStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new InitializeShareGroupStateResponseData.PartitionResult()
@ -582,7 +582,7 @@ class ShareCoordinatorServiceTest {
.setTopicId(topicId1)
.setPartitions(List.of(new InitializeShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)))
));
);
assertEquals(expectedResult, result);
}
@ -890,7 +890,7 @@ class ShareCoordinatorServiceTest {
HashSet<WriteShareGroupStateResponseData.WriteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = new HashSet<>(List.of(
Set<WriteShareGroupStateResponseData.WriteStateResult> expectedResult = Set.of(
new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new WriteShareGroupStateResponseData.PartitionResult()
@ -902,7 +902,7 @@ class ShareCoordinatorServiceTest {
.setPartitions(List.of(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
.setErrorMessage("Share coordinator is not available."))));
assertEquals(expectedResult, result);
}
@ -954,7 +954,7 @@ class ShareCoordinatorServiceTest {
HashSet<ReadShareGroupStateResponseData.ReadStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = new HashSet<>(List.of(
Set<ReadShareGroupStateResponseData.ReadStateResult> expectedResult = Set.of(
new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new ReadShareGroupStateResponseData.PartitionResult()
@ -966,7 +966,7 @@ class ShareCoordinatorServiceTest {
.setPartitions(List.of(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
.setErrorMessage("Share coordinator is not available."))));
assertEquals(expectedResult, result);
}
@ -1018,7 +1018,7 @@ class ShareCoordinatorServiceTest {
HashSet<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> expectedResult = new HashSet<>(List.of(
Set<ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult> expectedResult = Set.of(
new ReadShareGroupStateSummaryResponseData.ReadStateSummaryResult()
.setTopicId(topicId2)
.setPartitions(List.of(new ReadShareGroupStateSummaryResponseData.PartitionResult()
@ -1030,7 +1030,7 @@ class ShareCoordinatorServiceTest {
.setPartitions(List.of(new ReadShareGroupStateSummaryResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
.setErrorMessage("Share coordinator is not available."))));
assertEquals(expectedResult, result);
}
@ -1080,7 +1080,7 @@ class ShareCoordinatorServiceTest {
HashSet<DeleteShareGroupStateResponseData.DeleteStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<DeleteShareGroupStateResponseData.DeleteStateResult> expectedResult = new HashSet<>(List.of(
Set<DeleteShareGroupStateResponseData.DeleteStateResult> expectedResult = Set.of(
new DeleteShareGroupStateResponseData.DeleteStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult()
@ -1092,7 +1092,7 @@ class ShareCoordinatorServiceTest {
.setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
.setErrorMessage("Share coordinator is not available."))));
assertEquals(expectedResult, result);
}
@ -1141,7 +1141,7 @@ class ShareCoordinatorServiceTest {
HashSet<InitializeShareGroupStateResponseData.InitializeStateResult> result = new HashSet<>(future.get(5, TimeUnit.SECONDS).results());
HashSet<InitializeShareGroupStateResponseData.InitializeStateResult> expectedResult = new HashSet<>(List.of(
Set<InitializeShareGroupStateResponseData.InitializeStateResult> expectedResult = Set.of(
new InitializeShareGroupStateResponseData.InitializeStateResult()
.setTopicId(topicId2)
.setPartitions(List.of(new InitializeShareGroupStateResponseData.PartitionResult()
@ -1153,7 +1153,7 @@ class ShareCoordinatorServiceTest {
.setPartitions(List.of(new InitializeShareGroupStateResponseData.PartitionResult()
.setPartition(partition1)
.setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
.setErrorMessage("Share coordinator is not available.")))));
.setErrorMessage("Share coordinator is not available."))));
assertEquals(expectedResult, result);
}
@ -2249,12 +2249,12 @@ class ShareCoordinatorServiceTest {
}
private void checkMetrics(Metrics metrics) {
Set<MetricName> usualMetrics = new HashSet<>(List.of(
Set<MetricName> usualMetrics = Set.of(
metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP)
));
);
usualMetrics.forEach(metric -> assertTrue(metrics.metrics().containsKey(metric)));
}

View File

@ -26,9 +26,8 @@ import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_LATENCY_SENSOR_NAME;
import static org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME;
@ -42,12 +41,12 @@ public class ShareCoordinatorMetricsTest {
public void testMetricNames() {
Metrics metrics = new Metrics();
HashSet<MetricName> expectedMetrics = new HashSet<>(List.of(
Set<MetricName> expectedMetrics = Set.of(
metrics.metricName("write-rate", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-total", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-avg", ShareCoordinatorMetrics.METRICS_GROUP),
metrics.metricName("write-latency-max", ShareCoordinatorMetrics.METRICS_GROUP)
));
);
ShareCoordinatorMetrics coordMetrics = new ShareCoordinatorMetrics(metrics);
for (MetricName metricName : expectedMetrics) {

View File

@ -44,7 +44,6 @@ import java.nio.file.StandardOpenOption;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -607,7 +606,7 @@ public class ProducerStateManagerTest {
appendClientEntry(stateManager, producerId, epoch, 2, 2L, false);
stateManager.takeSnapshot();
assertEquals(2, Objects.requireNonNull(logDir.listFiles()).length);
assertEquals(new HashSet<>(List.of(2L, 3L)), currentSnapshotOffsets());
assertEquals(Set.of(2L, 3L), currentSnapshotOffsets());
stateManager.deleteSnapshotsBefore(3L);
assertEquals(1, Objects.requireNonNull(logDir.listFiles()).length);
@ -629,7 +628,7 @@ public class ProducerStateManagerTest {
appendClientEntry(stateManager, producerId, epoch, 2, 2L, false);
stateManager.takeSnapshot();
assertEquals(2, Objects.requireNonNull(logDir.listFiles()).length);
assertEquals(new HashSet<>(List.of(2L, 3L)), currentSnapshotOffsets());
assertEquals(Set.of(2L, 3L), currentSnapshotOffsets());
stateManager.truncateFullyAndStartAt(0L);
@ -655,7 +654,7 @@ public class ProducerStateManagerTest {
appendClientEntry(stateManager, producerId, epoch, 4, 4L, false);
stateManager.takeSnapshot();
assertEquals(2, Objects.requireNonNull(logDir.listFiles()).length);
assertEquals(new HashSet<>(List.of(3L, 5L)), currentSnapshotOffsets());
assertEquals(Set.of(3L, 5L), currentSnapshotOffsets());
// Truncate to the range (3, 5), this will delete the earlier snapshot until offset 3.
stateManager.truncateAndReload(3, 5, time.milliseconds());

View File

@ -96,7 +96,7 @@ public class KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest {
@Override
public Optional<Set<Integer>> partitions(final String topic, final String key, final Void value, final int numPartitions) {
return Optional.of(new HashSet<>(Arrays.asList(0, 1, 2)));
return Optional.of(Set.of(0, 1, 2));
}
}

View File

@ -747,7 +747,7 @@ public class QueryableStateIntegrationTest {
new KeyValue<>(keys[4], 2L))
);
final Set<KeyValue<String, Long>> expectedBatch1 =
new HashSet<>(Collections.singleton(new KeyValue<>(keys[4], 2L)));
Set.of(new KeyValue<>(keys[4], 2L));
IntegrationTestUtils.produceKeyValuesSynchronously(
streamOne,
@ -919,7 +919,7 @@ public class QueryableStateIntegrationTest {
new KeyValue<>(keys[4], "2"))
);
final Set<KeyValue<String, Long>> expectedBatch1 =
new HashSet<>(Collections.singleton(new KeyValue<>(keys[4], 2L)));
Set.of(new KeyValue<>(keys[4], 2L));
IntegrationTestUtils.produceKeyValuesSynchronously(
streamOne,

View File

@ -25,9 +25,7 @@ import org.apache.kafka.streams.state.internals.CompositeReadOnlySessionStore;
import org.apache.kafka.streams.state.internals.CompositeReadOnlyWindowStore;
import org.apache.kafka.streams.state.internals.StateStoreProvider;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
/**
@ -129,9 +127,9 @@ public final class QueryableStoreTypes {
extends QueryableStoreTypeMatcher<ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>>> {
TimestampedKeyValueStoreType() {
super(new HashSet<>(Arrays.asList(
super(Set.of(
TimestampedKeyValueStore.class,
ReadOnlyKeyValueStore.class)));
ReadOnlyKeyValueStore.class));
}
@Override
@ -158,9 +156,9 @@ public final class QueryableStoreTypes {
extends QueryableStoreTypeMatcher<ReadOnlyWindowStore<K, ValueAndTimestamp<V>>> {
TimestampedWindowStoreType() {
super(new HashSet<>(Arrays.asList(
super(Set.of(
TimestampedWindowStore.class,
ReadOnlyWindowStore.class)));
ReadOnlyWindowStore.class));
}
@Override

View File

@ -68,7 +68,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -2275,7 +2274,7 @@ public class TopologyTest {
private TopologyDescription.Source addSource(final String sourceName,
final String... sourceTopic) {
topology.addSource((AutoOffsetReset) null, sourceName, null, null, null, sourceTopic);
return new InternalTopologyBuilder.Source(sourceName, new HashSet<>(Arrays.asList(sourceTopic)), null);
return new InternalTopologyBuilder.Source(sourceName, Set.of(sourceTopic), null);
}
@SuppressWarnings("deprecation")
@ -2325,7 +2324,7 @@ public class TopologyTest {
topology.connectProcessorAndStateStores(processorName, storeNames);
}
final TopologyDescription.Processor expectedProcessorNode =
new InternalTopologyBuilder.Processor(processorName, new HashSet<>(Arrays.asList(storeNames)));
new InternalTopologyBuilder.Processor(processorName, Set.of(storeNames));
for (final TopologyDescription.Node parent : parents) {
((InternalTopologyBuilder.AbstractNode) parent).addSuccessor(expectedProcessorNode);

View File

@ -66,10 +66,8 @@ import org.mockito.Mockito;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
@ -585,7 +583,7 @@ public class KStreamKStreamJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -707,7 +705,7 @@ public class KStreamKStreamJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -830,7 +828,7 @@ public class KStreamKStreamJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -1395,7 +1393,7 @@ public class KStreamKStreamJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -1662,7 +1660,7 @@ public class KStreamKStreamJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -39,9 +39,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
@ -457,7 +455,7 @@ public class KStreamKStreamLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -658,7 +656,7 @@ public class KStreamKStreamLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -821,7 +819,7 @@ public class KStreamKStreamLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -893,7 +891,7 @@ public class KStreamKStreamLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -49,9 +49,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
import java.util.concurrent.atomic.AtomicReference;
@ -472,7 +470,7 @@ public class KStreamKStreamOuterJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -538,7 +536,7 @@ public class KStreamKStreamOuterJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -666,7 +664,7 @@ public class KStreamKStreamOuterJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =
@ -779,7 +777,7 @@ public class KStreamKStreamOuterJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), PROPS)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -46,9 +46,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
@ -335,7 +333,7 @@ public class KStreamKTableJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(streamTopic, tableTopic)), copartitionGroups.iterator().next());
assertEquals(Set.of(streamTopic, tableTopic), copartitionGroups.iterator().next());
}
@Test

View File

@ -40,9 +40,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Random;
import java.util.Set;
@ -119,7 +117,7 @@ public class KStreamKTableLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(streamTopic, tableTopic)), copartitionGroups.iterator().next());
assertEquals(Set.of(streamTopic, tableTopic), copartitionGroups.iterator().next());
}
@Test

View File

@ -42,9 +42,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
@ -374,7 +372,7 @@ public class KTableKTableInnerJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -50,9 +50,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Locale;
import java.util.Properties;
import java.util.Random;
@ -93,7 +91,7 @@ public class KTableKTableLeftJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -40,9 +40,7 @@ import org.junit.jupiter.api.Test;
import java.time.Duration;
import java.time.Instant;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Properties;
import java.util.Set;
@ -79,7 +77,7 @@ public class KTableKTableOuterJoinTest {
TopologyWrapper.getInternalTopologyBuilder(builder.build()).copartitionGroups();
assertEquals(1, copartitionGroups.size());
assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next());
assertEquals(Set.of(topic1, topic2), copartitionGroups.iterator().next());
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
final TestInputTopic<Integer, String> inputTopic1 =

View File

@ -328,7 +328,7 @@ public class InternalTopicManagerTest {
final InternalTopicManager internalTopicManager =
new InternalTopicManager(time, mockAdminClient, new StreamsConfig(config));
try {
final Set<String> topic1set = new HashSet<>(Collections.singletonList(topic1));
final Set<String> topic1set = Set.of(topic1);
internalTopicManager.getTopicPartitionInfo(topic1set, null);
} catch (final TimeoutException expected) {
@ -338,7 +338,7 @@ public class InternalTopicManagerTest {
mockAdminClient.timeoutNextRequest(1);
try {
final Set<String> topic2set = new HashSet<>(Collections.singletonList(topic2));
final Set<String> topic2set = Set.of(topic2);
internalTopicManager.getTopicPartitionInfo(topic2set, null);
} catch (final TimeoutException expected) {
@ -353,7 +353,7 @@ public class InternalTopicManagerTest {
final InternalTopicManager internalTopicManager =
new InternalTopicManager(time, mockAdminClient, new StreamsConfig(config));
try {
final Set<String> topic1set = new HashSet<>(Collections.singletonList(topic1));
final Set<String> topic1set = Set.of(topic1);
final Set<String> topic2set = new HashSet<>(Collections.singletonList(topic2));
internalTopicManager.getNumPartitions(topic1set, topic2set);
@ -365,7 +365,7 @@ public class InternalTopicManagerTest {
mockAdminClient.timeoutNextRequest(1);
try {
final Set<String> topic1set = new HashSet<>(Collections.singletonList(topic1));
final Set<String> topic1set = Set.of(topic1);
final Set<String> topic2set = new HashSet<>(Collections.singletonList(topic2));
internalTopicManager.getNumPartitions(topic1set, topic2set);

View File

@ -202,7 +202,7 @@ public class ProcessorNodeTest {
throw new TaskCorruptedException(tasksIds, new InvalidOffsetException("Invalid offset") {
@Override
public Set<TopicPartition> partitions() {
return new HashSet<>(Collections.singletonList(new TopicPartition("topic", 0)));
return Set.of(new TopicPartition("topic", 0));
}
});
}

View File

@ -1991,7 +1991,7 @@ public class StreamTaskTest {
task = new StreamTask(
taskId,
new HashSet<>(List.of(partition1, repartition)),
Set.of(partition1, repartition),
topology,
consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(),
@ -2920,7 +2920,7 @@ public class StreamTaskTest {
final TaskCorruptedException expectedException = new TaskCorruptedException(tasksIds, new InvalidOffsetException("Invalid offset") {
@Override
public Set<TopicPartition> partitions() {
return new HashSet<>(Collections.singletonList(new TopicPartition("topic", 0)));
return Set.of(new TopicPartition("topic", 0));
}
});
@ -3084,7 +3084,7 @@ public class StreamTaskTest {
return new StreamTask(
taskId,
new HashSet<>(List.of(partition1)),
Set.of(partition1),
topology,
consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(),
@ -3234,7 +3234,7 @@ public class StreamTaskTest {
return new StreamTask(
taskId,
new HashSet<>(List.of(partition1)),
Set.of(partition1),
topology,
consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(),
@ -3340,7 +3340,7 @@ public class StreamTaskTest {
task = new StreamTask(
taskId,
new HashSet<>(List.of(partition1)),
Set.of(partition1),
topology,
consumer,
new TopologyConfig(null, config, new Properties()).getTaskConfig(),

View File

@ -893,8 +893,8 @@ public class StreamsPartitionAssignorTest {
// then metadata gets populated
assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment();
// check assigned partitions
assertEquals(Set.of(new HashSet<>(List.of(t1p0, t2p0, t1p0, t2p0, t1p1, t2p1, t1p2, t2p2))),
Set.of(new HashSet<>(assignments.get("consumer10").partitions())));
assertEquals(Set.of(t1p0, t2p0, t1p1, t2p1, t1p2, t2p2),
new HashSet<>(assignments.get("consumer10").partitions()));
// the first consumer
info10 = checkAssignment(allTopics, assignments.get("consumer10"));

View File

@ -21,7 +21,6 @@ import org.apache.kafka.streams.processor.TaskId;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
@ -35,7 +34,7 @@ import static org.mockito.Mockito.when;
public class TaskExecutionMetadataTest {
static final String TOPOLOGY1 = "topology1";
static final String TOPOLOGY2 = "topology2";
static final Set<String> NAMED_TOPOLOGIES = new HashSet<>(Arrays.asList(TOPOLOGY1, TOPOLOGY2));
static final Set<String> NAMED_TOPOLOGIES = Set.of(TOPOLOGY1, TOPOLOGY2);
static final int TIME_ZERO = 0;
static final int CONSTANT_BACKOFF_MS = 5000;

View File

@ -753,7 +753,7 @@ public class LegacyStickyTaskAssignorTest {
public void shouldMoveMinimalNumberOfTasksWhenPreviouslyAboveCapacityAndNewClientAdded(final String rackAwareStrategy) {
setUp(rackAwareStrategy);
final Set<TaskId> p1PrevTasks = new HashSet<>(List.of(TASK_0_0, TASK_0_2));
final Set<TaskId> p2PrevTasks = new HashSet<>(List.of(TASK_0_1, TASK_0_3));
final Set<TaskId> p2PrevTasks = Set.of(TASK_0_1, TASK_0_3);
createClientWithPreviousActiveTasks(PID_1, 1, TASK_0_0, TASK_0_2);
createClientWithPreviousActiveTasks(PID_2, 1, TASK_0_1, TASK_0_3);

View File

@ -23,8 +23,6 @@ import org.apache.kafka.streams.processor.internals.Task;
import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@ -58,13 +56,13 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.fail;
public class SubscriptionInfoTest {
private static final Set<TaskId> ACTIVE_TASKS = new HashSet<>(Arrays.asList(
private static final Set<TaskId> ACTIVE_TASKS = Set.of(
TASK_0_0,
TASK_0_1,
TASK_1_0));
private static final Set<TaskId> STANDBY_TASKS = new HashSet<>(Arrays.asList(
TASK_1_0);
private static final Set<TaskId> STANDBY_TASKS = Set.of(
TASK_1_1,
TASK_2_0));
TASK_2_0);
private static final Map<TaskId, Long> TASK_OFFSET_SUMS = mkMap(
mkEntry(TASK_0_0, Task.LATEST_OFFSET),
mkEntry(TASK_0_1, Task.LATEST_OFFSET),

View File

@ -33,8 +33,6 @@ import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
@ -175,7 +173,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
// while for TimeOrderedWindowStores, actualFrom = observedStreamTime - retention
// expired record
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
@ -191,7 +189,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
// actualFrom = 150000, hence not ignored
if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
@ -199,32 +197,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} else {
assertEquals(
new HashSet<>(Collections.singletonList("one")),
Set.of("one"),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE))));
}
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
@ -241,13 +239,13 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
@ -264,7 +262,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals(
// expired record
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
@ -272,32 +270,32 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} else {
assertEquals(
// expired record
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
}
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("six")),
Set.of("six"),
valuesToSetAndCloseIterator(windowStore.fetch(
6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
@ -314,50 +312,50 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals(
// expired record
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("six")),
Set.of("six"),
valuesToSetAndCloseIterator(windowStore.fetch(
6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("seven")),
Set.of("seven"),
valuesToSetAndCloseIterator(windowStore.fetch(
7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
@ -374,25 +372,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
@ -400,7 +398,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals(
// expired record
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
@ -408,7 +406,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
} else {
assertEquals(
// expired record
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
@ -416,25 +414,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
}
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("six")),
Set.of("six"),
valuesToSetAndCloseIterator(windowStore.fetch(
6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("seven")),
Set.of("seven"),
valuesToSetAndCloseIterator(windowStore.fetch(
7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("eight")),
Set.of("eight"),
valuesToSetAndCloseIterator(windowStore.fetch(
8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
@ -616,55 +614,55 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
// while for TimeOrderedWindowStores, actualFrom = observedStreamTime - retention
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 3 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
@ -678,25 +676,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
context.restore(STORE_NAME, changeLog);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(startTime - WINDOW_SIZE),
ofEpochMilli(startTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
1,
ofEpochMilli(startTime + increment - WINDOW_SIZE),
ofEpochMilli(startTime + increment + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(startTime + increment * 2 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 2 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
3,
ofEpochMilli(startTime + increment * 3 - WINDOW_SIZE),
@ -712,14 +710,14 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
// actualFrom = 240,000, hence not ignored
if (storeType() == StoreType.RocksDBWindowStore) {
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 4 + WINDOW_SIZE))));
} else {
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(
4,
ofEpochMilli(startTime + increment * 4 - WINDOW_SIZE),
@ -727,25 +725,25 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
}
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(
5,
ofEpochMilli(startTime + increment * 5 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 5 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("six")),
Set.of("six"),
valuesToSetAndCloseIterator(windowStore.fetch(
6,
ofEpochMilli(startTime + increment * 6 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 6 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("seven")),
Set.of("seven"),
valuesToSetAndCloseIterator(windowStore.fetch(
7,
ofEpochMilli(startTime + increment * 7 - WINDOW_SIZE),
ofEpochMilli(startTime + increment * 7 + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("eight")),
Set.of("eight"),
valuesToSetAndCloseIterator(windowStore.fetch(
8,
ofEpochMilli(startTime + increment * 8 - WINDOW_SIZE),
@ -786,7 +784,7 @@ public abstract class AbstractRocksDBWindowStoreTest extends AbstractWindowBytes
private Set<String> segmentDirs(final File baseDir) {
final File windowDir = new File(baseDir, windowStore.name());
return new HashSet<>(asList(requireNonNull(windowDir.list())));
return Set.of(requireNonNull(windowDir.list()));
}
}

View File

@ -566,43 +566,43 @@ public abstract class AbstractSessionBytesStoreTest {
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions("a", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 3L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 3L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions("aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(2L, 4L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(2L, 4L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions("a", "aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions("a", "aa", 10, 0)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(Collections.singletonList(2L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(2L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions(null, "aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions("a", null, 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.findSessions(null, null, 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
}
@ -622,43 +622,43 @@ public abstract class AbstractSessionBytesStoreTest {
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions("a", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 3L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 3L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions("aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(2L, 4L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(2L, 4L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions("a", "aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions("a", "aa", 10, 0)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(Collections.singletonList(2L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(2L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions(null, "aa", 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions("a", null, 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
try (final KeyValueIterator<Windowed<String>, Long> iterator =
sessionStore.backwardFindSessions(null, null, 0, Long.MAX_VALUE)
) {
assertThat(valuesToSet(iterator), equalTo(new HashSet<>(asList(1L, 2L, 3L, 4L, 5L))));
assertThat(valuesToSet(iterator), equalTo(Set.of(1L, 2L, 3L, 4L, 5L)));
}
}
@ -961,11 +961,11 @@ public abstract class AbstractSessionBytesStoreTest {
sessionStore.findSessions("a", "b", 0L, Long.MAX_VALUE)
) {
if (storeType() == StoreType.InMemoryStore) {
assertEquals(valuesToSet(iterator), new HashSet<>(Arrays.asList(2L, 3L, 4L)));
assertEquals(valuesToSet(iterator), Set.of(2L, 3L, 4L));
} else {
// The 2 records with values 2L and 3L are considered expired as
// their end times < observed stream time - retentionPeriod + 1.
assertEquals(valuesToSet(iterator), new HashSet<>(Collections.singletonList(4L)));
assertEquals(valuesToSet(iterator), Set.of(4L));
}
}
}

View File

@ -129,7 +129,7 @@ public abstract class AbstractWindowBytesStoreTest {
putFirstBatch(windowStore, defaultStartTime, context);
assertEquals(
new HashSet<>(Collections.singletonList("zero")),
Set.of("zero"),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(defaultStartTime - WINDOW_SIZE),
@ -145,91 +145,91 @@ public abstract class AbstractWindowBytesStoreTest {
assertEquals("two+6", windowStore.fetch(2, defaultStartTime + 8L));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime - 2L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1")),
Set.of("two", "two+1"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")),
Set.of("two", "two+1", "two+2"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
Set.of("two", "two+1", "two+2", "two+3"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4")),
Set.of("two", "two+1", "two+2", "two+3", "two+4"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5")),
Set.of("two", "two+1", "two+2", "two+3", "two+4", "two+5"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6")),
Set.of("two", "two+1", "two+2", "two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4", "two+5", "two+6")),
Set.of("two+1", "two+2", "two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5", "two+6")),
Set.of("two+2", "two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
Set.of("two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")),
Set.of("two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+5", "two+6")),
Set.of("two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("two+6")),
Set.of("two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
2,
ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE),
@ -561,70 +561,70 @@ public abstract class AbstractWindowBytesStoreTest {
putFirstBatch(windowStore, defaultStartTime, context);
assertEquals(
new HashSet<>(Collections.singletonList("zero")),
Set.of("zero"),
valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime))));
assertEquals(
new HashSet<>(Collections.singletonList("one")),
Set.of("one"),
valuesToSetAndCloseIterator(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L))));
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L))));
assertEquals(
new HashSet<>(Collections.singletonList("three")),
Set.of("three"),
valuesToSetAndCloseIterator(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L))));
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L))));
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L))));
putSecondBatch(windowStore, defaultStartTime);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime - 1L))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - WINDOW_SIZE), ofEpochMilli(defaultStartTime))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 1L))));
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 2L))));
assertEquals(
new HashSet<>(asList("two", "two+1")),
Set.of("two", "two+1"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 3L))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")),
Set.of("two", "two+1", "two+2"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 4L))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
Set.of("two", "two+1", "two+2", "two+3"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 5L))));
assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")),
Set.of("two+1", "two+2", "two+3", "two+4"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 6L))));
assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")),
Set.of("two+2", "two+3", "two+4", "two+5"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 7L))));
assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
Set.of("two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 8L))));
assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")),
Set.of("two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 9L))));
assertEquals(
new HashSet<>(asList("two+5", "two+6")),
Set.of("two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 10L))));
assertEquals(
new HashSet<>(Collections.singletonList("two+6")),
Set.of("two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 11L))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 12L))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 13L - WINDOW_SIZE), ofEpochMilli(defaultStartTime + 13L))));
// Flush the store and verify all current entries were properly flushed ...
@ -650,90 +650,90 @@ public abstract class AbstractWindowBytesStoreTest {
putFirstBatch(windowStore, defaultStartTime, context);
assertEquals(
new HashSet<>(Collections.singletonList("zero")),
Set.of("zero"),
valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime),
ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("one")),
Set.of("one"),
valuesToSetAndCloseIterator(windowStore.fetch(1, ofEpochMilli(defaultStartTime + 1L),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(3, ofEpochMilli(defaultStartTime + 3L),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("four")),
Set.of("four"),
valuesToSetAndCloseIterator(windowStore.fetch(4, ofEpochMilli(defaultStartTime + 4L),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("five")),
Set.of("five"),
valuesToSetAndCloseIterator(windowStore.fetch(5, ofEpochMilli(defaultStartTime + 5L),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
putSecondBatch(windowStore, defaultStartTime);
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 2L),
ofEpochMilli(defaultStartTime - 2L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("two")),
Set.of("two"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime - 1L),
ofEpochMilli(defaultStartTime - 1L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1")),
Set.of("two", "two+1"),
valuesToSetAndCloseIterator(windowStore
.fetch(2, ofEpochMilli(defaultStartTime), ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2")),
Set.of("two", "two+1", "two+2"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 1L),
ofEpochMilli(defaultStartTime + 1L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two", "two+1", "two+2", "two+3")),
Set.of("two", "two+1", "two+2", "two+3"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 2L),
ofEpochMilli(defaultStartTime + 2L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+1", "two+2", "two+3", "two+4")),
Set.of("two+1", "two+2", "two+3", "two+4"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 3L),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+2", "two+3", "two+4", "two+5")),
Set.of("two+2", "two+3", "two+4", "two+5"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 4L),
ofEpochMilli(defaultStartTime + 4L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+3", "two+4", "two+5", "two+6")),
Set.of("two+3", "two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 5L),
ofEpochMilli(defaultStartTime + 5L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+4", "two+5", "two+6")),
Set.of("two+4", "two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 6L),
ofEpochMilli(defaultStartTime + 6L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(asList("two+5", "two+6")),
Set.of("two+5", "two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 7L),
ofEpochMilli(defaultStartTime + 7L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.singletonList("two+6")),
Set.of("two+6"),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 8L),
ofEpochMilli(defaultStartTime + 8L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 9L),
ofEpochMilli(defaultStartTime + 9L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 10L),
ofEpochMilli(defaultStartTime + 10L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 11L),
ofEpochMilli(defaultStartTime + 11L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(2, ofEpochMilli(defaultStartTime + 12L),
ofEpochMilli(defaultStartTime + 12L + WINDOW_SIZE))));
@ -767,7 +767,7 @@ public abstract class AbstractWindowBytesStoreTest {
windowStore.put(0, "zero", defaultStartTime);
assertEquals(
new HashSet<>(Collections.singletonList("zero")),
Set.of("zero"),
valuesToSetAndCloseIterator(windowStore.fetch(0, ofEpochMilli(defaultStartTime - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + WINDOW_SIZE))));
@ -800,7 +800,7 @@ public abstract class AbstractWindowBytesStoreTest {
ofEpochMilli(defaultStartTime + 3L - WINDOW_SIZE),
ofEpochMilli(defaultStartTime + 3L + WINDOW_SIZE))));
assertEquals(
new HashSet<>(Collections.emptyList()),
Set.of(),
valuesToSetAndCloseIterator(windowStore.fetch(
0,
ofEpochMilli(defaultStartTime + 4L - WINDOW_SIZE),
@ -852,7 +852,7 @@ public abstract class AbstractWindowBytesStoreTest {
windowStore.put("aa", "0004", 1);
windowStore.put("a", "0005", 0x7a00000000000000L - 1);
final Set<String> expected = new HashSet<>(asList("0001", "0003", "0005"));
final Set<String> expected = Set.of("0001", "0003", "0005");
assertThat(
valuesToSetAndCloseIterator(windowStore.fetch("a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expected)
@ -862,20 +862,20 @@ public abstract class AbstractWindowBytesStoreTest {
toSet(windowStore.fetch("a", "a", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
assertThat(
set,
equalTo(new HashSet<>(asList(
equalTo(Set.of(
windowedPair("a", "0001", 0, windowSize),
windowedPair("a", "0003", 1, windowSize),
windowedPair("a", "0005", 0x7a00000000000000L - 1, windowSize)
)))
))
);
set = toSet(windowStore.fetch("aa", "aa", ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE)));
assertThat(
set,
equalTo(new HashSet<>(asList(
equalTo(Set.of(
windowedPair("aa", "0002", 0, windowSize),
windowedPair("aa", "0004", 1, windowSize)
)))
))
);
windowStore.close();
}
@ -934,17 +934,17 @@ public abstract class AbstractWindowBytesStoreTest {
windowStore.put(key2, "8", 59999);
windowStore.put(key3, "9", 59999);
final Set<String> expectedKey1 = new HashSet<>(asList("1", "4", "7"));
final Set<String> expectedKey1 = Set.of("1", "4", "7");
assertThat(
valuesToSetAndCloseIterator(windowStore.fetch(key1, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey1)
);
final Set<String> expectedKey2 = new HashSet<>(asList("2", "5", "8"));
final Set<String> expectedKey2 = Set.of("2", "5", "8");
assertThat(
valuesToSetAndCloseIterator(windowStore.fetch(key2, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey2)
);
final Set<String> expectedKey3 = new HashSet<>(asList("3", "6", "9"));
final Set<String> expectedKey3 = Set.of("3", "6", "9");
assertThat(
valuesToSetAndCloseIterator(windowStore.fetch(key3, ofEpochMilli(0), ofEpochMilli(Long.MAX_VALUE))),
equalTo(expectedKey3)

View File

@ -22,8 +22,7 @@ import org.apache.kafka.streams.state.KeyValueIterator;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import static org.apache.kafka.test.StreamsTestUtils.valuesToSet;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -48,7 +47,7 @@ public class InMemorySessionStoreTest extends AbstractSessionBytesStoreTest {
// Advance stream time to expire the first three record
sessionStore.put(new Windowed<>("aa", new SessionWindow(100, 2 * RETENTION_PERIOD)), 4L);
assertEquals(valuesToSet(iterator), new HashSet<>(Arrays.asList(1L, 2L, 3L, 4L)));
assertEquals(valuesToSet(iterator), Set.of(1L, 2L, 3L, 4L));
assertFalse(iterator.hasNext());
iterator.close();

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