Merge branch 'apache:trunk' into KAFKA-17103

This commit is contained in:
Chang-Yu Huang 2025-08-19 21:33:23 -04:00 committed by GitHub
commit 5817251104
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
120 changed files with 1892 additions and 2442 deletions

View File

@ -18,7 +18,6 @@ package org.apache.kafka.clients.consumer;
import org.apache.kafka.common.TopicPartition;
import java.time.Duration;
import java.util.Collection;
/**
@ -121,7 +120,7 @@ public interface ConsumerRebalanceListener {
/**
* A callback method the user can implement to provide handling of offset commits to a customized store.
* This method will be called during a rebalance operation when the consumer has to give up some partitions.
* It can also be called when consumer is being closed ({@link KafkaConsumer#close(Duration)})
* It can also be called when consumer is being closed ({@link KafkaConsumer#close(CloseOptions option)})
* or is unsubscribing ({@link KafkaConsumer#unsubscribe()}).
* It is recommended that offsets should be committed in this callback to either Kafka or a
* custom offset store to prevent duplicate data.

View File

@ -24,7 +24,6 @@ import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.function.BiConsumer;
@ -151,46 +150,6 @@ public class PartitionStates<S> {
}
}
public static class PartitionState<S> {
private final TopicPartition topicPartition;
private final S value;
public PartitionState(TopicPartition topicPartition, S state) {
this.topicPartition = Objects.requireNonNull(topicPartition);
this.value = Objects.requireNonNull(state);
}
public S value() {
return value;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
PartitionState<?> that = (PartitionState<?>) o;
return topicPartition.equals(that.topicPartition) && value.equals(that.value);
}
@Override
public int hashCode() {
int result = topicPartition.hashCode();
result = 31 * result + value.hashCode();
return result;
}
public TopicPartition topicPartition() {
return topicPartition;
}
@Override
public String toString() {
return "PartitionState(" + topicPartition + "=" + value + ')';
}
}
}

View File

@ -27,7 +27,7 @@
// the request is now relevant. Partitions will be processed in the order
// they appear in the request.
//
// Version 4 adds IsolationLevel. Starting in version 4, the reqestor must be
// Version 4 adds IsolationLevel. Starting in version 4, the requestor must be
// able to handle Kafka log message format version 2.
//
// Version 5 adds LogStartOffset to indicate the earliest available offset of

View File

@ -22,7 +22,6 @@ import org.apache.kafka.common.config.SaslConfigs;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -32,7 +31,7 @@ public class NoneConnectorClientConfigOverridePolicyTest extends BaseConnectorCl
@Test
public void testNoOverrides() {
testValidOverride(Collections.emptyMap());
testValidOverride(Map.of());
}
@Test

View File

@ -22,7 +22,6 @@ import org.apache.kafka.common.config.SaslConfigs;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -32,8 +31,7 @@ public class PrincipalConnectorClientConfigOverridePolicyTest extends BaseConnec
@Test
public void testPrincipalOnly() {
Map<String, Object> clientConfig = Collections.singletonMap(SaslConfigs.SASL_JAAS_CONFIG, "test");
testValidOverride(clientConfig);
testValidOverride(Map.of(SaslConfigs.SASL_JAAS_CONFIG, "test"));
}
@Test

View File

@ -25,7 +25,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -42,7 +42,7 @@ public class BooleanConverterTest {
@BeforeEach
public void setUp() {
converter.configure(Collections.emptyMap(), false);
converter.configure(Map.of(), false);
}
@Test

View File

@ -27,7 +27,7 @@ import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -42,7 +42,7 @@ public class ByteArrayConverterTest {
@BeforeEach
public void setUp() {
converter.configure(Collections.emptyMap(), false);
converter.configure(Map.of(), false);
}
@Test

View File

@ -48,7 +48,6 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -577,7 +576,7 @@ public class BlockingConnectorTest {
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
block.maybeBlockOn(CONNECTOR_TASK_CONFIGS);
return Collections.singletonList(Collections.emptyMap());
return List.of(Map.of());
}
@Override

View File

@ -58,7 +58,6 @@ import java.io.File;
import java.io.FileOutputStream;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -229,7 +228,7 @@ public class ConnectWorkerIntegrationTest {
// Restart the failed task
String taskRestartEndpoint = connect.endpointForResource(
String.format("connectors/%s/tasks/0/restart", CONNECTOR_NAME));
connect.requestPost(taskRestartEndpoint, "", Collections.emptyMap());
connect.requestPost(taskRestartEndpoint, "", Map.of());
// Ensure the task started successfully this time
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, numTasks,
@ -374,7 +373,7 @@ public class ConnectWorkerIntegrationTest {
NUM_TASKS, "Connector tasks did not start in time");
connector.awaitRecords(TimeUnit.MINUTES.toMillis(1));
// Then if we delete the connector, it and each of its tasks should be stopped by the framework
// Then, if we delete the connector, it and each of its tasks should be stopped by the framework
// even though the producer is blocked because there is no topic
StartAndStopLatch stopCounter = connector.expectedStops(1);
connect.deleteConnector(CONNECTOR_NAME);
@ -434,8 +433,8 @@ public class ConnectWorkerIntegrationTest {
"Connector did not stop in time"
);
// If the connector is truly stopped, we should also see an empty set of tasks and task configs
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Transition to RUNNING
connect.resumeConnector(CONNECTOR_NAME);
@ -463,8 +462,8 @@ public class ConnectWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector did not stop in time"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Transition to PAUSED
connect.pauseConnector(CONNECTOR_NAME);
@ -520,8 +519,8 @@ public class ConnectWorkerIntegrationTest {
"Connector did not stop in time"
);
// If the connector is truly stopped, we should also see an empty set of tasks and task configs
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Can resume a connector after its Connector has failed before shutdown after receiving a stop request
props.remove("connector.start.inject.error");
@ -542,8 +541,8 @@ public class ConnectWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector did not stop in time"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Can resume a connector after its Connector has failed during shutdown after receiving a stop request
connect.resumeConnector(CONNECTOR_NAME);
@ -580,8 +579,8 @@ public class ConnectWorkerIntegrationTest {
0,
"Connector was not created in a paused state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that a connector created in the PAUSED state can be resumed successfully
connect.resumeConnector(CONNECTOR_NAME);
@ -615,16 +614,16 @@ public class ConnectWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector was not created in a stopped state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that the offsets can be modified for a source connector created in the STOPPED state
// Alter the offsets so that only 5 messages are produced
connect.alterSourceConnectorOffset(
CONNECTOR_NAME,
Collections.singletonMap("task.id", CONNECTOR_NAME + "-0"),
Collections.singletonMap("saved", 5L)
Map.of("task.id", CONNECTOR_NAME + "-0"),
Map.of("saved", 5L)
);
// Verify that a connector created in the STOPPED state can be resumed successfully
@ -669,8 +668,8 @@ public class ConnectWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector was not created in a stopped state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that the offsets can be modified for a sink connector created in the STOPPED state
@ -726,8 +725,8 @@ public class ConnectWorkerIntegrationTest {
0,
"Connector was not created in a paused state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that a connector created in the PAUSED state can be deleted successfully
connect.deleteConnector(CONNECTOR_NAME);
@ -747,8 +746,8 @@ public class ConnectWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector was not created in a stopped state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that a connector created in the STOPPED state can be deleted successfully
connect.deleteConnector(CONNECTOR_NAME);
@ -1014,7 +1013,7 @@ public class ConnectWorkerIntegrationTest {
// an existing set of task configs that was written before the cluster was upgraded
try (JsonConverter converter = new JsonConverter()) {
converter.configure(
Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"),
Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"),
false
);
@ -1326,7 +1325,7 @@ public class ConnectWorkerIntegrationTest {
"Connector did not start or task did not fail in time"
);
assertEquals(
new ConnectorOffsets(Collections.emptyList()),
new ConnectorOffsets(List.of()),
connect.connectorOffsets(CONNECTOR_NAME),
"Connector should not have any committed offsets when only task fails on first record"
);
@ -1346,9 +1345,9 @@ public class ConnectWorkerIntegrationTest {
Map<String, Object> expectedOffsetKey = new HashMap<>();
expectedOffsetKey.put(SinkUtils.KAFKA_TOPIC_KEY, topic);
expectedOffsetKey.put(SinkUtils.KAFKA_PARTITION_KEY, 0);
Map<String, Object> expectedOffsetValue = Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 1);
Map<String, Object> expectedOffsetValue = Map.of(SinkUtils.KAFKA_OFFSET_KEY, 1);
ConnectorOffset expectedOffset = new ConnectorOffset(expectedOffsetKey, expectedOffsetValue);
ConnectorOffsets expectedOffsets = new ConnectorOffsets(Collections.singletonList(expectedOffset));
ConnectorOffsets expectedOffsets = new ConnectorOffsets(List.of(expectedOffset));
// Wait for it to commit offsets, signaling that it has successfully processed the record we produced earlier
waitForCondition(
@ -1443,7 +1442,7 @@ public class ConnectWorkerIntegrationTest {
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
return IntStream.range(0, maxTasks)
.mapToObj(i -> Collections.<String, String>emptyMap())
.mapToObj(i -> Map.<String, String>of())
.collect(Collectors.toList());
}

View File

@ -23,14 +23,12 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
@ -283,8 +281,8 @@ public class ConnectorHandle {
List<StartAndStopLatch> taskLatches = includeTasks
? taskHandles.values().stream()
.map(task -> task.expectedStarts(expectedStarts))
.collect(Collectors.toList())
: Collections.emptyList();
.toList()
: List.of();
return startAndStopCounter.expectedStarts(expectedStarts, taskLatches);
}
@ -292,8 +290,8 @@ public class ConnectorHandle {
List<StartAndStopLatch> taskLatches = includeTasks
? taskHandles.values().stream()
.map(task -> task.expectedStarts(expectedTasksStarts.get(task.taskId())))
.collect(Collectors.toList())
: Collections.emptyList();
.toList()
: List.of();
return startAndStopCounter.expectedStarts(expectedStarts, taskLatches);
}
@ -345,8 +343,8 @@ public class ConnectorHandle {
List<StartAndStopLatch> taskLatches = includeTasks
? taskHandles.values().stream()
.map(task -> task.expectedStops(expectedStops))
.collect(Collectors.toList())
: Collections.emptyList();
.toList()
: List.of();
return startAndStopCounter.expectedStops(expectedStops, taskLatches);
}
@ -354,8 +352,8 @@ public class ConnectorHandle {
List<StartAndStopLatch> taskLatches = includeTasks
? taskHandles.values().stream()
.map(task -> task.expectedStops(expectedTasksStops.get(task.taskId())))
.collect(Collectors.toList())
: Collections.emptyList();
.toList()
: List.of();
return startAndStopCounter.expectedStops(expectedStops, taskLatches);
}

View File

@ -30,7 +30,6 @@ import org.junit.jupiter.api.TestInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
@ -131,7 +130,7 @@ public class ConnectorRestartApiIntegrationTest {
// Call the Restart API
String restartEndpoint = connect.endpointForResource(
String.format("connectors/%s/restart", connectorName));
Response response = connect.requestPost(restartEndpoint, "", Collections.emptyMap());
Response response = connect.requestPost(restartEndpoint, "", Map.of());
assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus());
}
@ -152,7 +151,7 @@ public class ConnectorRestartApiIntegrationTest {
// Call the Restart API
String restartEndpoint = connect.endpointForResource(
String.format("connectors/%s/restart?onlyFailed=" + onlyFailed + "&includeTasks=" + includeTasks, connectorName));
Response response = connect.requestPost(restartEndpoint, "", Collections.emptyMap());
Response response = connect.requestPost(restartEndpoint, "", Map.of());
assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus());
}
@ -213,7 +212,7 @@ public class ConnectorRestartApiIntegrationTest {
@Test
public void testOneFailedTasksRestartOnlyOneTasks() throws Exception {
Set<String> tasksToFail = Collections.singleton(taskId(1));
Set<String> tasksToFail = Set.of(taskId(1));
failedTasksRestart(true, true, 0, buildExpectedTasksRestarts(tasksToFail), tasksToFail, false);
}

View File

@ -34,8 +34,6 @@ import org.junit.jupiter.api.Test;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -43,7 +41,6 @@ import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
@ -115,7 +112,7 @@ public class ConnectorTopicsIntegrationTest {
connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS);
connect.kafka().createTopic(BAR_TOPIC, NUM_TOPIC_PARTITIONS);
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.emptyList(),
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(),
"Active topic set is not empty for connector: " + FOO_CONNECTOR);
// start a source connector
@ -124,8 +121,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.singletonList(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
// start another source connector
connect.configureConnector(BAR_CONNECTOR, defaultSourceConnectorProps(BAR_TOPIC));
@ -133,8 +130,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(BAR_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, Collections.singletonList(BAR_TOPIC),
"Active topic set is not: " + Collections.singletonList(BAR_TOPIC) + " for connector: " + BAR_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, List.of(BAR_TOPIC),
"Active topic set is not: " + List.of(BAR_TOPIC) + " for connector: " + BAR_CONNECTOR);
// start a sink connector
connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC, BAR_TOPIC));
@ -142,8 +139,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Arrays.asList(FOO_TOPIC, BAR_TOPIC),
"Active topic set is not: " + Arrays.asList(FOO_TOPIC, BAR_TOPIC) + " for connector: " + SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC, BAR_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC, BAR_TOPIC) + " for connector: " + SINK_CONNECTOR);
// deleting a connector resets its active topics
connect.deleteConnector(BAR_CONNECTOR);
@ -151,7 +148,7 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorDoesNotExist(BAR_CONNECTOR,
"Connector wasn't deleted in time.");
connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, Collections.emptyList(),
connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, List.of(),
"Active topic set is not empty for deleted connector: " + BAR_CONNECTOR);
// Unfortunately there's currently no easy way to know when the consumer caught up with
@ -162,8 +159,8 @@ public class ConnectorTopicsIntegrationTest {
// reset active topics for the sink connector after one of the topics has become idle
connect.resetConnectorTopics(SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
}
@Test
@ -177,7 +174,7 @@ public class ConnectorTopicsIntegrationTest {
connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS);
connect.kafka().createTopic(BAR_TOPIC, NUM_TOPIC_PARTITIONS);
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.emptyList(),
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(),
"Active topic set is not empty for connector: " + FOO_CONNECTOR);
// start a source connector
@ -186,8 +183,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.singletonList(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
// start a sink connector
connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC));
@ -195,8 +192,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
// deleting a connector resets its active topics
connect.deleteConnector(FOO_CONNECTOR);
@ -204,7 +201,7 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorDoesNotExist(FOO_CONNECTOR,
"Connector wasn't deleted in time.");
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.emptyList(),
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(),
"Active topic set is not empty for deleted connector: " + FOO_CONNECTOR);
// Unfortunately there's currently no easy way to know when the consumer caught up with
@ -216,8 +213,8 @@ public class ConnectorTopicsIntegrationTest {
Exception e = assertThrows(ConnectRestException.class, () -> connect.resetConnectorTopics(SINK_CONNECTOR));
assertTrue(e.getMessage().contains("Topic tracking reset is disabled."));
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
}
@Test
@ -252,7 +249,7 @@ public class ConnectorTopicsIntegrationTest {
public void assertNoTopicStatusInStatusTopic() {
String statusTopic = workerProps.get(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG);
Consumer<byte[], byte[]> verifiableConsumer = connect.kafka().createConsumer(
Collections.singletonMap("group.id", "verifiable-consumer-group-0"));
Map.of("group.id", "verifiable-consumer-group-0"));
List<PartitionInfo> partitionInfos = verifiableConsumer.partitionsFor(statusTopic);
if (partitionInfos.isEmpty()) {
@ -260,7 +257,7 @@ public class ConnectorTopicsIntegrationTest {
}
List<TopicPartition> partitions = partitionInfos.stream()
.map(info -> new TopicPartition(info.topic(), info.partition()))
.collect(Collectors.toList());
.toList();
verifiableConsumer.assign(partitions);
// Based on the implementation of {@link org.apache.kafka.connect.util.KafkaBasedLog#readToLogEnd}

View File

@ -67,7 +67,6 @@ import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
@ -306,7 +305,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -366,7 +365,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -427,7 +426,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -538,7 +537,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -601,7 +600,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -664,7 +663,7 @@ public class ExactlyOnceSourceIntegrationTest {
String topic = "test-topic";
try (Admin admin = connect.kafka().createAdminClient()) {
admin.createTopics(Collections.singleton(new NewTopic(topic, 3, (short) 1))).all().get();
admin.createTopics(Set.of(new NewTopic(topic, 3, (short) 1))).all().get();
}
Map<String, String> props = new HashMap<>();
@ -690,7 +689,7 @@ public class ExactlyOnceSourceIntegrationTest {
// Grant the connector's admin permissions to access the topics for its records and offsets
// Intentionally leave out permissions required for fencing
try (Admin admin = connect.kafka().createAdminClient()) {
admin.createAcls(Arrays.asList(
admin.createAcls(List.of(
new AclBinding(
new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL),
new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW)
@ -737,7 +736,7 @@ public class ExactlyOnceSourceIntegrationTest {
// Now grant the necessary permissions for fencing to the connector's admin
try (Admin admin = connect.kafka().createAdminClient()) {
admin.createAcls(Arrays.asList(
admin.createAcls(List.of(
new AclBinding(
new ResourcePattern(ResourceType.TRANSACTIONAL_ID, Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, 0), PatternType.LITERAL),
new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW)
@ -864,7 +863,7 @@ public class ExactlyOnceSourceIntegrationTest {
.consume(
MINIMUM_MESSAGES,
TimeUnit.MINUTES.toMillis(1),
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
"test-topic")
.count();
assertTrue(recordNum >= MINIMUM_MESSAGES,
@ -874,7 +873,7 @@ public class ExactlyOnceSourceIntegrationTest {
ConsumerRecords<byte[], byte[]> offsetRecords = connectorTargetedCluster
.consumeAll(
TimeUnit.MINUTES.toMillis(1),
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
offsetsTopic
);
@ -930,7 +929,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> sourceRecords = connectorTargetedCluster.consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
topic
);
@ -939,7 +938,7 @@ public class ExactlyOnceSourceIntegrationTest {
// also have to check which offsets have actually been committed, since we no longer have exactly-once semantics
offsetRecords = connectorTargetedCluster.consumeAll(
CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null,
offsetsTopic
);
@ -999,13 +998,13 @@ public class ExactlyOnceSourceIntegrationTest {
private List<Long> parseAndAssertOffsetsForSingleTask(ConsumerRecords<byte[], byte[]> offsetRecords) {
Map<Integer, List<Long>> parsedOffsets = parseOffsetForTasks(offsetRecords);
assertEquals(Collections.singleton(0), parsedOffsets.keySet(), "Expected records to only be produced from a single task");
assertEquals(Set.of(0), parsedOffsets.keySet(), "Expected records to only be produced from a single task");
return parsedOffsets.get(0);
}
private List<Long> parseAndAssertValuesForSingleTask(ConsumerRecords<byte[], byte[]> sourceRecords) {
Map<Integer, List<Long>> parsedValues = parseValuesForTasks(sourceRecords);
assertEquals(Collections.singleton(0), parsedValues.keySet(), "Expected records to only be produced from a single task");
assertEquals(Set.of(0), parsedValues.keySet(), "Expected records to only be produced from a single task");
return parsedValues.get(0);
}
@ -1024,7 +1023,7 @@ public class ExactlyOnceSourceIntegrationTest {
parsedValues.replaceAll((task, values) -> {
Long committedValue = lastCommittedValues.get(task);
assertNotNull(committedValue, "No committed offset found for task " + task);
return values.stream().filter(v -> v <= committedValue).collect(Collectors.toList());
return values.stream().filter(v -> v <= committedValue).toList();
});
assertSeqnos(parsedValues, numTasks);
}
@ -1102,7 +1101,7 @@ public class ExactlyOnceSourceIntegrationTest {
JsonConverter offsetsConverter = new JsonConverter();
// The JSON converter behaves identically for keys and values. If that ever changes, we may need to update this test to use
// separate converter instances.
offsetsConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false);
offsetsConverter.configure(Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), false);
Map<Integer, List<Long>> result = new HashMap<>();
for (ConsumerRecord<byte[], byte[]> offsetRecord : offsetRecords) {
@ -1284,7 +1283,7 @@ public class ExactlyOnceSourceIntegrationTest {
// Request a read to the end of the offsets topic
context.offsetStorageReader().offset(Collections.singletonMap("", null));
// Produce a record to the offsets topic
return Collections.singletonList(new SourceRecord(null, null, topic, null, "", null, null));
return List.of(new SourceRecord(null, null, topic, null, "", null, null));
}
@Override

View File

@ -28,7 +28,6 @@ import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
@ -286,15 +285,15 @@ public class InternalTopicsIntegrationTest {
}
protected Map<String, String> compactCleanupPolicy() {
return Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT);
return Map.of(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT);
}
protected Map<String, String> deleteCleanupPolicy() {
return Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE);
return Map.of(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_DELETE);
}
protected Map<String, String> noTopicSettings() {
return Collections.emptyMap();
return Map.of();
}
protected Map<String, String> compactAndDeleteCleanupPolicy() {

View File

@ -44,7 +44,6 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -107,7 +106,7 @@ public class OffsetsApiIntegrationTest {
}
try {
assertEquals(
Collections.emptySet(),
Set.of(),
remainingConnectors,
"Some connectors were not properly cleaned up after this test"
);
@ -150,11 +149,11 @@ public class OffsetsApiIntegrationTest {
}
private static EmbeddedConnectCluster defaultConnectCluster() {
return createOrReuseConnectWithWorkerProps(Collections.emptyMap());
return createOrReuseConnectWithWorkerProps(Map.of());
}
private static EmbeddedConnectCluster exactlyOnceSourceConnectCluster() {
Map<String, String> workerProps = Collections.singletonMap(
Map<String, String> workerProps = Map.of(
DistributedConfig.EXACTLY_ONCE_SOURCE_SUPPORT_CONFIG,
"enabled"
);
@ -288,8 +287,8 @@ public class OffsetsApiIntegrationTest {
@Test
public void testAlterOffsetsNonExistentConnector() {
ConnectRestException e = assertThrows(ConnectRestException.class,
() -> connect.alterConnectorOffsets("non-existent-connector", new ConnectorOffsets(Collections.singletonList(
new ConnectorOffset(Collections.emptyMap(), Collections.emptyMap())))));
() -> connect.alterConnectorOffsets("non-existent-connector", new ConnectorOffsets(List.of(
new ConnectorOffset(Map.of(), Map.of())))));
assertEquals(404, e.errorCode());
}
@ -304,8 +303,7 @@ public class OffsetsApiIntegrationTest {
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsets.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
Collections.singletonMap("saved", 5))
new ConnectorOffset(Map.of("task.id", connectorName + "-" + i), Map.of("saved", 5))
);
}
@ -403,7 +401,7 @@ public class OffsetsApiIntegrationTest {
partition = new HashMap<>();
partition.put(SinkUtils.KAFKA_TOPIC_KEY, topic);
partition.put(SinkUtils.KAFKA_PARTITION_KEY, i);
offsetsToAlter.add(new ConnectorOffset(partition, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 5)));
offsetsToAlter.add(new ConnectorOffset(partition, Map.of(SinkUtils.KAFKA_OFFSET_KEY, 5)));
}
// Alter the sink connector's offsets, with retry logic (since we just stopped the connector)
@ -425,7 +423,7 @@ public class OffsetsApiIntegrationTest {
partition = new HashMap<>();
partition.put(SinkUtils.KAFKA_TOPIC_KEY, topic);
partition.put(SinkUtils.KAFKA_PARTITION_KEY, i);
offsetsToAlter.add(new ConnectorOffset(partition, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 3)));
offsetsToAlter.add(new ConnectorOffset(partition, Map.of(SinkUtils.KAFKA_OFFSET_KEY, 3)));
}
response = connect.alterConnectorOffsets(connectorName, new ConnectorOffsets(offsetsToAlter));
@ -475,7 +473,7 @@ public class OffsetsApiIntegrationTest {
Map<String, Object> partition = new HashMap<>();
partition.put(SinkUtils.KAFKA_TOPIC_KEY, topic);
partition.put(SinkUtils.KAFKA_PARTITION_KEY, 0);
List<ConnectorOffset> offsetsToAlter = Collections.singletonList(new ConnectorOffset(partition, null));
List<ConnectorOffset> offsetsToAlter = List.of(new ConnectorOffset(partition, null));
ConnectRestException e = assertThrows(ConnectRestException.class,
() -> connect.alterConnectorOffsets(connectorName, new ConnectorOffsets(offsetsToAlter)));
@ -602,8 +600,7 @@ public class OffsetsApiIntegrationTest {
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsetsToAlter.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
Collections.singletonMap("saved", 5))
new ConnectorOffset(Map.of("task.id", connectorName + "-" + i), Map.of("saved", 5))
);
}
@ -623,8 +620,7 @@ public class OffsetsApiIntegrationTest {
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsetsToAlter.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
Collections.singletonMap("saved", 7))
new ConnectorOffset(Map.of("task.id", connectorName + "-" + i), Map.of("saved", 7))
);
}

View File

@ -30,8 +30,8 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
@ -350,8 +350,7 @@ public class RebalanceSourceConnectorsIntegrationTest {
private static String formatAssignment(Map<String, Collection<String>> assignment) {
StringBuilder result = new StringBuilder();
for (String worker : assignment.keySet().stream().sorted().toList()) {
result.append(String.format("\n%s=%s", worker, assignment.getOrDefault(worker,
Collections.emptyList())));
result.append(String.format("\n%s=%s", worker, assignment.getOrDefault(worker, List.of())));
}
return result.toString();
}

View File

@ -31,7 +31,6 @@ import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@ -112,7 +111,7 @@ public class RestExtensionIntegrationTest {
workerId,
null
),
Collections.singletonMap(
Map.of(
0,
new TaskState(0, "RUNNING", workerId, null)
),

View File

@ -63,7 +63,6 @@ import java.nio.charset.StandardCharsets;
import java.security.GeneralSecurityException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -199,7 +198,7 @@ public class RestForwardingIntegrationTest {
.putConnectorConfig(any(), any(), isNull(), anyBoolean(), followerCallbackCaptor.capture());
// Leader will reply
ConnectorInfo connectorInfo = new ConnectorInfo("blah", Collections.emptyMap(), Collections.emptyList(), ConnectorType.SOURCE);
ConnectorInfo connectorInfo = new ConnectorInfo("blah", Map.of(), List.of(), ConnectorType.SOURCE);
Herder.Created<ConnectorInfo> leaderAnswer = new Herder.Created<>(true, connectorInfo);
ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> leaderCallbackCaptor = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> {

View File

@ -28,10 +28,10 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Properties;
@ -98,10 +98,10 @@ public class SinkConnectorsIntegrationTest {
public void testEagerConsumerPartitionAssignment() throws Exception {
final String topic1 = "topic1", topic2 = "topic2", topic3 = "topic3";
final TopicPartition tp1 = new TopicPartition(topic1, 0), tp2 = new TopicPartition(topic2, 0), tp3 = new TopicPartition(topic3, 0);
final Collection<String> topics = Arrays.asList(topic1, topic2, topic3);
final Collection<String> topics = List.of(topic1, topic2, topic3);
Map<String, String> connectorProps = baseSinkConnectorProps(String.join(",", topics));
// Need an eager assignor here; round robin is as good as any
// Need an eager assignor here; round-robin is as good as any
connectorProps.put(
CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
RoundRobinAssignor.class.getName());
@ -205,7 +205,7 @@ public class SinkConnectorsIntegrationTest {
public void testCooperativeConsumerPartitionAssignment() throws Exception {
final String topic1 = "topic1", topic2 = "topic2", topic3 = "topic3";
final TopicPartition tp1 = new TopicPartition(topic1, 0), tp2 = new TopicPartition(topic2, 0), tp3 = new TopicPartition(topic3, 0);
final Collection<String> topics = Arrays.asList(topic1, topic2, topic3);
final Collection<String> topics = List.of(topic1, topic2, topic3);
Map<String, String> connectorProps = baseSinkConnectorProps(String.join(",", topics));
connectorProps.put(

View File

@ -25,7 +25,6 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
@ -150,7 +149,7 @@ public class SourceConnectorsIntegrationTest {
// start the clusters
connect.start();
connect.kafka().createTopic(BAR_TOPIC, DEFAULT_PARTITIONS, DEFAULT_REPLICATION_FACTOR, Collections.emptyMap());
connect.kafka().createTopic(BAR_TOPIC, DEFAULT_PARTITIONS, DEFAULT_REPLICATION_FACTOR, Map.of());
connect.assertions().assertTopicsExist(BAR_TOPIC);
connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR,

View File

@ -30,7 +30,6 @@ import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -38,7 +37,6 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import jakarta.ws.rs.core.Response;
@ -102,7 +100,7 @@ public class StandaloneWorkerIntegrationTest {
StandaloneWorkerIntegrationTest::isModified
);
assertEquals(
Collections.emptyMap(),
Map.of(),
invalidModifiedLoggers,
"No loggers should have a non-null last-modified timestamp"
);
@ -155,9 +153,9 @@ public class StandaloneWorkerIntegrationTest {
assertTrue(affectedLoggers.contains(namespace));
List<String> invalidAffectedLoggers = affectedLoggers.stream()
.filter(l -> !l.startsWith(namespace))
.collect(Collectors.toList());
.toList();
assertEquals(
Collections.emptyList(),
List.of(),
invalidAffectedLoggers,
"No loggers outside the namespace '" + namespace
+ "' should have been included in the response for a request to modify that namespace"
@ -188,7 +186,7 @@ public class StandaloneWorkerIntegrationTest {
)
);
assertEquals(
Collections.emptyMap(),
Map.of(),
invalidAffectedLoggerLevels,
"At least one logger in the affected namespace '" + namespace
+ "' does not have the expected level of '" + level
@ -199,7 +197,7 @@ public class StandaloneWorkerIntegrationTest {
Set<String> droppedLoggers = Utils.diff(HashSet::new, initialLevels.keySet(), newLevels.keySet());
assertEquals(
Collections.emptySet(),
Set.of(),
droppedLoggers,
"At least one logger was present in the listing of all loggers "
+ "before the logging level for namespace '" + namespace
@ -212,7 +210,7 @@ public class StandaloneWorkerIntegrationTest {
e -> !hasNamespace(e, namespace) && !e.getValue().equals(initialLevels.get(e.getKey()))
);
assertEquals(
Collections.emptyMap(),
Map.of(),
invalidUnaffectedLoggerLevels,
"At least one logger outside of the affected namespace '" + namespace
+ "' has a different logging level or last-modified timestamp than it did "
@ -256,8 +254,8 @@ public class StandaloneWorkerIntegrationTest {
CONNECTOR_NAME,
"Connector was not created in a stopped state"
);
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME));
assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that a connector created in the STOPPED state can be resumed successfully
connect.resumeConnector(CONNECTOR_NAME);

View File

@ -25,7 +25,6 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
@ -92,7 +91,7 @@ public class StartAndStopLatchTest {
@Test
public void shouldReturnFalseWhenAwaitingForDependentLatchToComplete() throws Throwable {
StartAndStopLatch depLatch = new StartAndStopLatch(1, 1, this::complete, null, clock);
dependents = Collections.singletonList(depLatch);
dependents = List.of(depLatch);
latch = new StartAndStopLatch(1, 1, this::complete, dependents, clock);
future = asyncAwait(100);
@ -106,7 +105,7 @@ public class StartAndStopLatchTest {
@Test
public void shouldReturnTrueWhenAwaitingForStartAndStopAndDependentLatch() throws Throwable {
StartAndStopLatch depLatch = new StartAndStopLatch(1, 1, this::complete, null, clock);
dependents = Collections.singletonList(depLatch);
dependents = List.of(depLatch);
latch = new StartAndStopLatch(1, 1, this::complete, dependents, clock);
future = asyncAwait(100);

View File

@ -33,13 +33,11 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.LongStream;
/**
@ -190,7 +188,7 @@ public class TestableSourceConnector extends SampleSourceConnector {
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
Map<String, Object> offset = Optional.ofNullable(
context.offsetStorageReader().offset(sourcePartition(taskId)))
.orElse(Collections.emptyMap());
.orElse(Map.of());
startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L);
seqno = startingSeqno;
log.info("Started {} task {} with properties {}", this.getClass().getSimpleName(), taskId, props);
@ -235,7 +233,7 @@ public class TestableSourceConnector extends SampleSourceConnector {
maybeDefineTransactionBoundary(record);
return record;
})
.collect(Collectors.toList());
.toList();
}
return null;
}
@ -295,10 +293,10 @@ public class TestableSourceConnector extends SampleSourceConnector {
}
public static Map<String, Object> sourcePartition(String taskId) {
return Collections.singletonMap("task.id", taskId);
return Map.of("task.id", taskId);
}
public static Map<String, Object> sourceOffset(long seqno) {
return Collections.singletonMap("saved", seqno);
return Map.of("saved", seqno);
}
}

View File

@ -34,7 +34,6 @@ import java.util.Map;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import static java.util.Collections.singletonMap;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.PREDICATES_CONFIG;
@ -173,7 +172,7 @@ public class TransformationIntegrationTest {
connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS);
// Assert that we didn't see any baz
Map<String, Long> expectedRecordCounts = singletonMap(fooTopic, (long) numFooRecords);
Map<String, Long> expectedRecordCounts = Map.of(fooTopic, (long) numFooRecords);
assertObservedRecords(observedRecords, expectedRecordCounts);
// delete connector
@ -253,7 +252,7 @@ public class TransformationIntegrationTest {
// wait for the connector tasks to commit all records.
connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS);
Map<String, Long> expectedRecordCounts = singletonMap(topic, (long) (numRecords / 2));
Map<String, Long> expectedRecordCounts = Map.of(topic, (long) (numRecords / 2));
assertObservedRecords(observedRecords, expectedRecordCounts);
// delete connector

View File

@ -73,9 +73,7 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
@ -156,27 +154,27 @@ public class AbstractHerderTest {
private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
Map.of(CONN1, 3),
Map.of(CONN1, CONN1_CONFIG),
Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Collections.emptySet(),
Collections.emptySet());
Map.of(),
Map.of(),
Map.of(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Set.of(),
Set.of());
private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Collections.emptySet(),
Collections.emptySet());
Map.of(CONN1, 3),
Map.of(CONN1, CONN1_CONFIG),
Map.of(CONN1, TargetState.STARTED),
Map.of(),
Map.of(),
Map.of(),
Map.of(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Set.of(),
Set.of());
private final String workerId = "workerId";
private final String kafkaClusterId = "I4ZmrWqfT2e-upky_4fdPA";
@ -198,7 +196,7 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder();
when(configStore.snapshot()).thenReturn(SNAPSHOT);
assertEquals(Collections.singleton(CONN1), new HashSet<>(herder.connectors()));
assertEquals(Set.of(CONN1), Set.copyOf(herder.connectors()));
}
@Test
@ -220,7 +218,7 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder();
when(herder.rawConfig(connectorName)).thenReturn(Collections.singletonMap(
when(herder.rawConfig(connectorName)).thenReturn(Map.of(
ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName()
));
@ -228,7 +226,7 @@ public class AbstractHerderTest {
.thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation));
when(statusStore.getAll(connectorName))
.thenReturn(Collections.singletonList(
.thenReturn(List.of(
new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation)));
ConnectorStateInfo state = herder.connectorStatus(connectorName);
@ -255,13 +253,13 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder();
when(herder.rawConfig(connectorName))
.thenReturn(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "missing"));
.thenReturn(Map.of(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "missing"));
when(statusStore.get(connectorName))
.thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation));
when(statusStore.getAll(connectorName))
.thenReturn(Collections.singletonList(
.thenReturn(List.of(
new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation)));
ConnectorStateInfo state = herder.connectorStatus(connectorName);
@ -292,7 +290,7 @@ public class AbstractHerderTest {
assertEquals(CONN1, info.name());
assertEquals(CONN1_CONFIG, info.config());
assertEquals(Arrays.asList(TASK0, TASK1, TASK2), info.tasks());
assertEquals(List.of(TASK0, TASK1, TASK2), info.tasks());
assertEquals(ConnectorType.SOURCE, info.type());
}
@ -332,7 +330,7 @@ public class AbstractHerderTest {
assertEquals(CONN1, info.name());
assertEquals(CONN1_CONFIG, info.config());
assertEquals(Arrays.asList(TASK0, TASK1, TASK2), info.tasks());
assertEquals(List.of(TASK0, TASK1, TASK2), info.tasks());
assertEquals(ConnectorType.UNKNOWN, info.type());
}
@ -468,8 +466,8 @@ public class AbstractHerderTest {
public void testConfigValidationEmptyConfig() {
AbstractHerder herder = createConfigValidationHerder(SampleSourceConnector.class, noneConnectorClientConfigOverridePolicy, 0);
assertThrows(BadRequestException.class, () -> herder.validateConnectorConfig(Collections.emptyMap(), s -> null, false));
verify(transformer).transform(Collections.emptyMap());
assertThrows(BadRequestException.class, () -> herder.validateConnectorConfig(Map.of(), s -> null, false));
verify(transformer).transform(Map.of());
assertEquals(worker.getPlugins(), plugins);
}
@ -478,13 +476,13 @@ public class AbstractHerderTest {
final Class<? extends Connector> connectorClass = SampleSourceConnector.class;
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy);
Map<String, String> config = Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName());
Map<String, String> config = Map.of(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName());
ConfigInfos result = herder.validateConnectorConfig(config, s -> null, false);
// We expect there to be errors due to the missing name and .... Note that these assertions depend heavily on
// the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name());
assertEquals(Arrays.asList(ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP,
assertEquals(List.of(ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP, ConnectorConfig.ERROR_GROUP,
SourceConnectorConfig.TOPIC_CREATION_GROUP, SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_GROUP,
SourceConnectorConfig.OFFSETS_TOPIC_GROUP), result.groups());
@ -572,7 +570,7 @@ public class AbstractHerderTest {
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy);
// 2 transform aliases defined -> 2 plugin lookups
Mockito.lenient().when(plugins.transformations()).thenReturn(Collections.singleton(transformationPluginDesc()));
Mockito.lenient().when(plugins.transformations()).thenReturn(Set.of(transformationPluginDesc()));
Mockito.lenient().when(plugins.newPlugin(SampleTransformation.class.getName(), null, classLoader)).thenReturn(new SampleTransformation());
// Define 2 transformations. One has a class defined and so can get embedded configs, the other is missing
@ -591,7 +589,7 @@ public class AbstractHerderTest {
// the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name());
// Each transform also gets its own group
List<String> expectedGroups = Arrays.asList(
List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP,
@ -626,8 +624,8 @@ public class AbstractHerderTest {
final Class<? extends Connector> connectorClass = SampleSourceConnector.class;
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy);
Mockito.lenient().when(plugins.transformations()).thenReturn(Collections.singleton(transformationPluginDesc()));
Mockito.lenient().when(plugins.predicates()).thenReturn(Collections.singleton(predicatePluginDesc()));
Mockito.lenient().when(plugins.transformations()).thenReturn(Set.of(transformationPluginDesc()));
Mockito.lenient().when(plugins.predicates()).thenReturn(Set.of(predicatePluginDesc()));
Mockito.lenient().when(plugins.newPlugin(SampleTransformation.class.getName(), null, classLoader)).thenReturn(new SampleTransformation());
Mockito.lenient().when(plugins.newPlugin(SamplePredicate.class.getName(), null, classLoader)).thenReturn(new SamplePredicate());
@ -650,7 +648,7 @@ public class AbstractHerderTest {
// the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name());
// Each transform also gets its own group
List<String> expectedGroups = Arrays.asList(
List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP,
@ -716,7 +714,7 @@ public class AbstractHerderTest {
// the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(SampleSourceConnector.class.getName(), result.name());
// Each transform also gets its own group
List<String> expectedGroups = Arrays.asList(
List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP,
@ -891,7 +889,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C");
List<String> groups = Arrays.asList("groupB", "group C");
List<String> groups = List.of("groupB", "group C");
List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1");
@ -922,7 +920,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C");
List<String> groups = Arrays.asList("groupB", "group C");
List<String> groups = List.of("groupB", "group C");
List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1");
@ -953,7 +951,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C");
List<String> groups = Arrays.asList("groupB", "group C");
List<String> groups = List.of("groupB", "group C");
List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1");
@ -1143,7 +1141,7 @@ public class AbstractHerderTest {
when(worker.getPlugins()).thenReturn(plugins);
when(plugins.newConnector(anyString(), any())).thenThrow(new ConnectException("No class found"));
AbstractHerder herder = testHerder();
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connName)));
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Map.of(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connName)));
}
@Test
@ -1155,7 +1153,7 @@ public class AbstractHerderTest {
@Test
public void testGetConnectorTypeWithEmptyConfig() {
AbstractHerder herder = testHerder();
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.emptyMap()));
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Map.of()));
}
@Test
@ -1170,9 +1168,9 @@ public class AbstractHerderTest {
@Test
public void testConnectorOffsets() throws Exception {
ConnectorOffsets offsets = new ConnectorOffsets(Arrays.asList(
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"))
ConnectorOffsets offsets = new ConnectorOffsets(List.of(
new ConnectorOffset(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
new ConnectorOffset(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"))
));
@SuppressWarnings("unchecked")
ArgumentCaptor<Callback<ConnectorOffsets>> workerCallback = ArgumentCaptor.forClass(Callback.class);
@ -1203,7 +1201,7 @@ public class AbstractHerderTest {
when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIG.size());
List<Map<String, String>> alteredTaskConfigs = new ArrayList<>(TASK_CONFIGS);
alteredTaskConfigs.set(alteredTaskConfigs.size() - 1, Collections.emptyMap());
alteredTaskConfigs.set(alteredTaskConfigs.size() - 1, Map.of());
// Last task config is different; should report a change
assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, alteredTaskConfigs));
@ -1220,15 +1218,15 @@ public class AbstractHerderTest {
ClusterConfigState snapshotWithNoAppliedConfig = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
Map.of(CONN1, 3),
Map.of(CONN1, CONN1_CONFIG),
Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet()
Map.of(),
Map.of(),
Map.of(),
Set.of(),
Set.of()
);
assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithNoAppliedConfig, CONN1, TASK_CONFIGS));
@ -1238,15 +1236,15 @@ public class AbstractHerderTest {
ClusterConfigState snapshotWithDifferentAppliedConfig = new ClusterConfigState(
1,
null,
Collections.singletonMap(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED),
Map.of(CONN1, 3),
Map.of(CONN1, CONN1_CONFIG),
Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(appliedConfig)),
Collections.emptySet(),
Collections.emptySet()
Map.of(),
Map.of(),
Map.of(CONN1, new AppliedConnectorConfig(appliedConfig)),
Set.of(),
Set.of()
);
assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithDifferentAppliedConfig, CONN1, TASK_CONFIGS));
}
@ -1254,12 +1252,12 @@ public class AbstractHerderTest {
protected void addConfigKey(Map<String, ConfigDef.ConfigKey> keys, String name, String group) {
ConfigDef configDef = new ConfigDef().define(name, ConfigDef.Type.STRING, null, null,
ConfigDef.Importance.HIGH, "doc", group, 10,
ConfigDef.Width.MEDIUM, "display name", Collections.emptyList(), null, null);
ConfigDef.Width.MEDIUM, "display name", List.of(), null, null);
keys.putAll(configDef.configKeys());
}
protected void addValue(List<ConfigValue> values, String name, String value, String... errors) {
values.add(new ConfigValue(name, value, new ArrayList<>(), Arrays.asList(errors)));
values.add(new ConfigValue(name, value, new ArrayList<>(), List.of(errors)));
}
protected void assertInfoKey(ConfigInfos infos, String name, String group) {
@ -1277,7 +1275,7 @@ public class AbstractHerderTest {
ConfigValueInfo info = findInfo(infos, name).configValue();
assertEquals(name, info.name());
assertEquals(value, info.value());
assertEquals(Arrays.asList(errors), info.errors());
assertEquals(List.of(errors), info.errors());
}
protected ConfigInfo findInfo(ConfigInfos infos, String name) {
@ -1293,7 +1291,7 @@ public class AbstractHerderTest {
}
private void testConfigProviderRegex(String rawConnConfig, boolean expected) {
Set<String> keys = keysWithVariableValues(Collections.singletonMap("key", rawConnConfig), ConfigTransformer.DEFAULT_PATTERN);
Set<String> keys = keysWithVariableValues(Map.of("key", rawConnConfig), ConfigTransformer.DEFAULT_PATTERN);
boolean actual = !keys.isEmpty() && keys.contains("key");
assertEquals(expected, actual, String.format("%s should have matched regex", rawConnConfig));
}

View File

@ -73,8 +73,6 @@ import org.mockito.stubbing.Answer;
import java.nio.ByteBuffer;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -121,8 +119,8 @@ public class AbstractWorkerSourceTaskTest {
private static final String TOPIC = "topic";
private static final String OTHER_TOPIC = "other-topic";
private static final Map<String, byte[]> PARTITION = Collections.singletonMap("key", "partition".getBytes());
private static final Map<String, Integer> OFFSET = Collections.singletonMap("key", 12);
private static final Map<String, byte[]> PARTITION = Map.of("key", "partition".getBytes());
private static final Map<String, Integer> OFFSET = Map.of("key", 12);
// Connect-format data
private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA;
@ -253,7 +251,7 @@ public class AbstractWorkerSourceTaskTest {
createWorkerTask();
// Can just use the same record for key and value
List<SourceRecord> records = Collections.singletonList(
List<SourceRecord> records = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)
);
@ -282,7 +280,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain();
expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList(
workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
);
workerTask.sendRecords();
@ -302,7 +300,7 @@ public class AbstractWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC);
expectApplyTransformationChain();
workerTask.toSend = Collections.singletonList(
workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
);
assertThrows(InvalidRecordException.class, workerTask::sendRecords);
@ -319,7 +317,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain();
expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList(
workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
);
workerTask.sendRecords();
@ -345,7 +343,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain();
expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList(
workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD,
null, connectHeaders)
);
@ -367,7 +365,7 @@ public class AbstractWorkerSourceTaskTest {
SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders();
createWorkerTask(stringConverter, testConverter, stringConverter, RetryWithToleranceOperatorTest.noneOperator(),
Collections::emptyList, transformationChain);
List::of, transformationChain);
expectSendRecord(null);
expectApplyTransformationChain();
@ -383,7 +381,7 @@ public class AbstractWorkerSourceTaskTest {
org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders()
.addString("encoding", encodingB);
workerTask.toSend = Arrays.asList(
workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a",
Schema.STRING_SCHEMA, stringA, null, headersA),
new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b",
@ -426,13 +424,13 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc));
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
expectSendRecord(emptyHeaders());
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
verifySendRecord(2);
@ -452,11 +450,11 @@ public class AbstractWorkerSourceTaskTest {
when(admin.describeTopics(TOPIC))
.thenThrow(new RetriableException(new TimeoutException("timeout")))
.thenReturn(Collections.emptyMap());
.thenReturn(Map.of());
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
assertEquals(Arrays.asList(record1, record2), workerTask.toSend);
assertEquals(List.of(record1, record2), workerTask.toSend);
verify(admin, never()).createOrFindTopics(any(NewTopic.class));
verifyNoMoreInteractions(admin);
@ -477,16 +475,16 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC);
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class)))
// First call to create the topic times out
.thenThrow(new RetriableException(new TimeoutException("timeout")))
// Next attempt succeeds
.thenReturn(createdTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
assertEquals(Arrays.asList(record1, record2), workerTask.toSend);
assertEquals(List.of(record1, record2), workerTask.toSend);
// Next they all succeed
workerTask.sendRecords();
@ -509,9 +507,9 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(OTHER_TOPIC);
when(admin.describeTopics(anyString()))
.thenReturn(Collections.emptyMap())
.thenReturn(Map.of())
.thenThrow(new RetriableException(new TimeoutException("timeout")))
.thenReturn(Collections.emptyMap());
.thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenAnswer(
(Answer<TopicAdmin.TopicCreationResponse>) invocation -> {
NewTopic newTopic = invocation.getArgument(0);
@ -519,9 +517,9 @@ public class AbstractWorkerSourceTaskTest {
});
// Try to send 3, make first pass, second fail. Should save last record
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
workerTask.sendRecords();
assertEquals(Collections.singletonList(record3), workerTask.toSend);
assertEquals(List.of(record3), workerTask.toSend);
// Next they all succeed
workerTask.sendRecords();
@ -532,10 +530,10 @@ public class AbstractWorkerSourceTaskTest {
ArgumentCaptor<NewTopic> newTopicCaptor = ArgumentCaptor.forClass(NewTopic.class);
verify(admin, times(2)).createOrFindTopics(newTopicCaptor.capture());
assertEquals(Arrays.asList(TOPIC, OTHER_TOPIC), newTopicCaptor.getAllValues()
assertEquals(List.of(TOPIC, OTHER_TOPIC), newTopicCaptor.getAllValues()
.stream()
.map(NewTopic::name)
.collect(Collectors.toList()));
.toList());
}
@Test
@ -550,16 +548,16 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC);
expectPreliminaryCalls(OTHER_TOPIC);
when(admin.describeTopics(anyString())).thenReturn(Collections.emptyMap());
when(admin.describeTopics(anyString())).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class)))
.thenReturn(createdTopic(TOPIC))
.thenThrow(new RetriableException(new TimeoutException("timeout")))
.thenReturn(createdTopic(OTHER_TOPIC));
// Try to send 3, make first pass, second fail. Should save last record
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
workerTask.sendRecords();
assertEquals(Collections.singletonList(record3), workerTask.toSend);
assertEquals(List.of(record3), workerTask.toSend);
verifyTopicCreation(2, TOPIC, OTHER_TOPIC); // Second call to createOrFindTopics will throw
// Next they all succeed
@ -581,7 +579,7 @@ public class AbstractWorkerSourceTaskTest {
new ConnectException(new TopicAuthorizationException("unauthorized"))
);
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords);
}
@ -593,12 +591,12 @@ public class AbstractWorkerSourceTaskTest {
SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
expectPreliminaryCalls(TOPIC);
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenThrow(
new ConnectException(new TopicAuthorizationException("unauthorized"))
);
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords);
verify(admin).createOrFindTopics(any());
@ -614,10 +612,10 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC);
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(TopicAdmin.EMPTY_CREATION);
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords);
verify(admin).createOrFindTopics(any());
@ -634,10 +632,10 @@ public class AbstractWorkerSourceTaskTest {
expectSendRecord(emptyHeaders());
expectApplyTransformationChain();
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(foundTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
ArgumentCaptor<ProducerRecord<byte[], byte[]>> sent = verifySendRecord(2);
@ -659,10 +657,10 @@ public class AbstractWorkerSourceTaskTest {
expectSendRecord(emptyHeaders());
expectApplyTransformationChain();
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(createdTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
ArgumentCaptor<ProducerRecord<byte[], byte[]>> sent = verifySendRecord(2);
@ -688,13 +686,13 @@ public class AbstractWorkerSourceTaskTest {
when(transformationChain.apply(any(), eq(record2))).thenReturn(null);
when(transformationChain.apply(any(), eq(record3))).thenReturn(record3);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc));
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
when(producer.send(any(), any())).thenThrow(new RetriableException("Retriable exception")).thenReturn(null);
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
// The first two records are filtered out / dropped by the transformation chain; only the third record will be attempted to be sent.
// The producer throws a RetriableException the first time we try to send the third record
@ -721,11 +719,11 @@ public class AbstractWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc));
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
workerTask.toSend = Arrays.asList(record1);
workerTask.toSend = List.of(record1);
// The transformation errored out so the error should be re-raised by sendRecords with error tolerance None
Exception exception = assertThrows(ConnectException.class, workerTask::sendRecords);
@ -752,7 +750,7 @@ public class AbstractWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC);
workerTask.toSend = Arrays.asList(record1);
workerTask.toSend = List.of(record1);
// The transformation errored out so the error should be ignored & the record skipped with error tolerance all
assertTrue(workerTask.sendRecords());
@ -778,11 +776,11 @@ public class AbstractWorkerSourceTaskTest {
// When we try to convert the key/value of each record, throw an exception
throwExceptionWhenConvertKey(emptyHeaders(), TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc));
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
// Send records should fail when errors.tolerance is none and the conversion call fails
Exception exception = assertThrows(ConnectException.class, workerTask::sendRecords);
@ -813,7 +811,7 @@ public class AbstractWorkerSourceTaskTest {
// When we try to convert the key/value of each record, throw an exception
throwExceptionWhenConvertKey(emptyHeaders(), TOPIC);
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
// With errors.tolerance to all, the failed conversion should simply skip the record, and record successful batch
assertTrue(workerTask.sendRecords());
@ -883,15 +881,15 @@ public class AbstractWorkerSourceTaskTest {
@SuppressWarnings("SameParameterValue")
private TopicAdmin.TopicCreationResponse createdTopic(String topic) {
Set<String> created = Collections.singleton(topic);
Set<String> existing = Collections.emptySet();
Set<String> created = Set.of(topic);
Set<String> existing = Set.of();
return new TopicAdmin.TopicCreationResponse(created, existing);
}
@SuppressWarnings("SameParameterValue")
private TopicAdmin.TopicCreationResponse foundTopic(String topic) {
Set<String> created = Collections.emptySet();
Set<String> existing = Collections.singleton(topic);
Set<String> created = Set.of();
Set<String> existing = Set.of(topic);
return new TopicAdmin.TopicCreationResponse(created, existing);
}
@ -945,13 +943,13 @@ public class AbstractWorkerSourceTaskTest {
}
private void createWorkerTask(TransformationChain transformationChain, RetryWithToleranceOperator toleranceOperator) {
createWorkerTask(keyConverter, valueConverter, headerConverter, toleranceOperator, Collections::emptyList,
createWorkerTask(keyConverter, valueConverter, headerConverter, toleranceOperator, List::of,
transformationChain);
}
private void createWorkerTask() {
createWorkerTask(
keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noneOperator(), Collections::emptyList, transformationChain);
keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noneOperator(), List::of, transformationChain);
}
private void createWorkerTask(Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter,

View File

@ -30,7 +30,6 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -53,7 +52,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap<>()) {
@Override
public Set<PluginDesc<Transformation<?>>> transformations() {
return Collections.emptySet();
return Set.of();
}
};
@ -472,7 +471,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
Plugins mockPlugins = mock(Plugins.class);
when(mockPlugins.newPlugin(HasDuplicateConfigTransformation.class.getName(),
null, (ClassLoader) null)).thenReturn(new HasDuplicateConfigTransformation());
when(mockPlugins.transformations()).thenReturn(Collections.emptySet());
when(mockPlugins.transformations()).thenReturn(Set.of());
ConfigDef def = ConnectorConfig.enrich(mockPlugins, new ConfigDef(), props, false);
assertEnrichedConfigDef(def, prefix, HasDuplicateConfigTransformation.MUST_EXIST_KEY, ConfigDef.Type.BOOLEAN);
assertEnrichedConfigDef(def, prefix, TransformationStage.PREDICATE_CONFIG, ConfigDef.Type.STRING);

View File

@ -76,14 +76,11 @@ import org.mockito.quality.Strictness;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonList;
import static org.apache.kafka.common.utils.Time.SYSTEM;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
@ -226,7 +223,7 @@ public class ErrorHandlingTaskTest {
LogReporter<ConsumerRecord<byte[], byte[]>> reporter = new LogReporter.Sink(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator = operator();
createSinkTask(initialState, retryWithToleranceOperator, singletonList(reporter));
createSinkTask(initialState, retryWithToleranceOperator, List.of(reporter));
// valid json
ConsumerRecord<byte[], byte[]> record1 = new ConsumerRecord<>(
@ -278,14 +275,14 @@ public class ErrorHandlingTaskTest {
LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator();
createSourceTask(initialState, retryWithToleranceOperator, singletonList(reporter));
createSourceTask(initialState, retryWithToleranceOperator, List.of(reporter));
// valid json
Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build();
Struct struct1 = new Struct(valSchema).put("val", 1234);
SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1);
SourceRecord record1 = new SourceRecord(Map.of(), Map.of(), TOPIC, PARTITION1, valSchema, struct1);
Struct struct2 = new Struct(valSchema).put("val", 6789);
SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2);
SourceRecord record2 = new SourceRecord(Map.of(), Map.of(), TOPIC, PARTITION1, valSchema, struct2);
when(workerSourceTask.isStopping())
.thenReturn(false)
@ -295,8 +292,8 @@ public class ErrorHandlingTaskTest {
doReturn(true).when(workerSourceTask).commitOffsets();
when(sourceTask.poll())
.thenReturn(singletonList(record1))
.thenReturn(singletonList(record2));
.thenReturn(List.of(record1))
.thenReturn(List.of(record2));
expectTopicCreation(TOPIC);
@ -340,14 +337,14 @@ public class ErrorHandlingTaskTest {
LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator();
createSourceTask(initialState, retryWithToleranceOperator, singletonList(reporter), badConverter());
createSourceTask(initialState, retryWithToleranceOperator, List.of(reporter), badConverter());
// valid json
Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build();
Struct struct1 = new Struct(valSchema).put("val", 1234);
SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1);
SourceRecord record1 = new SourceRecord(Map.of(), Map.of(), TOPIC, PARTITION1, valSchema, struct1);
Struct struct2 = new Struct(valSchema).put("val", 6789);
SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2);
SourceRecord record2 = new SourceRecord(Map.of(), Map.of(), TOPIC, PARTITION1, valSchema, struct2);
when(workerSourceTask.isStopping())
.thenReturn(false)
@ -357,8 +354,8 @@ public class ErrorHandlingTaskTest {
doReturn(true).when(workerSourceTask).commitOffsets();
when(sourceTask.poll())
.thenReturn(singletonList(record1))
.thenReturn(singletonList(record2));
.thenReturn(List.of(record1))
.thenReturn(List.of(record2));
expectTopicCreation(TOPIC);
workerSourceTask.initialize(TASK_CONFIG);
workerSourceTask.initializeAndStart();
@ -392,7 +389,7 @@ public class ErrorHandlingTaskTest {
private void verifyInitializeSink() {
verify(sinkTask).start(TASK_PROPS);
verify(sinkTask).initialize(any(WorkerSinkTaskContext.class));
verify(consumer).subscribe(eq(singletonList(TOPIC)),
verify(consumer).subscribe(eq(List.of(TOPIC)),
any(ConsumerRebalanceListener.class));
}
@ -410,9 +407,9 @@ public class ErrorHandlingTaskTest {
private void expectTopicCreation(String topic) {
if (enableTopicCreation) {
when(admin.describeTopics(topic)).thenReturn(Collections.emptyMap());
Set<String> created = Collections.singleton(topic);
Set<String> existing = Collections.emptySet();
when(admin.describeTopics(topic)).thenReturn(Map.of());
Set<String> created = Set.of(topic);
Set<String> existing = Set.of();
TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing);
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(response);
}
@ -428,7 +425,7 @@ public class ErrorHandlingTaskTest {
Plugin<Transformation<SinkRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SinkRecord>(), taskId, "test");
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> sinkTransforms =
new TransformationChain<>(singletonList(new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator);
new TransformationChain<>(List.of(new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator);
Plugin<Converter> keyConverterPlugin = metrics.wrap(converter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(converter, taskId, false);
@ -463,7 +460,7 @@ public class ErrorHandlingTaskTest {
private void createSourceTask(TargetState initialState, RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator,
List<ErrorReporter<SourceRecord>> errorReporters, Converter converter) {
Plugin<Transformation<SourceRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SourceRecord>(), taskId, "test");
TransformationChain<SourceRecord, SourceRecord> sourceTransforms = new TransformationChain<>(singletonList(
TransformationChain<SourceRecord, SourceRecord> sourceTransforms = new TransformationChain<>(List.of(
new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator);
Plugin<Converter> keyConverterPlugin = metrics.wrap(converter, taskId, true);

View File

@ -68,8 +68,6 @@ import org.mockito.stubbing.OngoingStubbing;
import org.mockito.verification.VerificationMode;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -84,7 +82,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import static java.util.Collections.emptySet;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
@ -119,7 +116,7 @@ import static org.mockito.Mockito.when;
@MockitoSettings(strictness = Strictness.WARN)
public class ExactlyOnceWorkerSourceTaskTest {
private static final String TOPIC = "topic";
private static final Map<String, byte[]> PARTITION = Collections.singletonMap("key", "partition".getBytes());
private static final Map<String, byte[]> PARTITION = Map.of("key", "partition".getBytes());
private static final Map<String, ?> OFFSET = offset(12);
// Connect-format data
@ -169,7 +166,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
private static final SourceRecord SOURCE_RECORD_2 =
new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, VALUE_2);
private static final List<SourceRecord> RECORDS = Arrays.asList(SOURCE_RECORD_1, SOURCE_RECORD_2);
private static final List<SourceRecord> RECORDS = List.of(SOURCE_RECORD_1, SOURCE_RECORD_2);
private final AtomicReference<CountDownLatch> pollLatch = new AtomicReference<>(new CountDownLatch(0));
private final AtomicReference<List<SourceRecord>> pollRecords = new AtomicReference<>(RECORDS);
@ -267,7 +264,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
}
private static Map<String, ?> offset(int n) {
return Collections.singletonMap("key", n);
return Map.of("key", n);
}
private void createWorkerTask() {
@ -285,7 +282,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin,
transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore,
config, clusterConfigState, metrics, errorHandlingMetrics, plugins.delegatingLoader(), time, RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore,
sourceConfig, Runnable::run, preProducerCheck, postProducerCheck, Collections::emptyList, null, TestPlugins.noOpLoaderSwap());
sourceConfig, Runnable::run, preProducerCheck, postProducerCheck, List::of, null, TestPlugins.noOpLoaderSwap());
}
@ParameterizedTest
@ -296,7 +293,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
workerTask.removeMetrics();
assertEquals(emptySet(), filterToTaskMetrics(metrics.metrics().metrics().keySet()));
assertEquals(Set.of(), filterToTaskMetrics(metrics.metrics().metrics().keySet()));
}
private Set<MetricName> filterToTaskMetrics(Set<MetricName> metricNames) {
@ -562,7 +559,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
createWorkerTask();
// Make sure the task returns empty batches from poll before we start polling it
pollRecords.set(Collections.emptyList());
pollRecords.set(List.of());
when(offsetWriter.beginFlush()).thenReturn(false);
@ -956,7 +953,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue();
// We're trying to send three records
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
OngoingStubbing<Future<RecordMetadata>> producerSend = when(producer.send(any(), any()));
// The first one is sent successfully
producerSend = expectSuccessfulSend(producerSend);
@ -966,7 +963,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
expectSuccessfulSend(producerSend);
assertFalse(workerTask.sendRecords());
assertEquals(Arrays.asList(record2, record3), workerTask.toSend);
assertEquals(List.of(record2, record3), workerTask.toSend);
verify(producer).beginTransaction();
// When using poll-based transaction boundaries, we do not commit transactions while retrying delivery for a batch
verify(producer, never()).commitTransaction();
@ -1001,7 +998,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
when(producer.send(any(), any()))
.thenThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC)));
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords);
verify(producer).beginTransaction();
@ -1080,7 +1077,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
}
private void awaitEmptyPolls(int minimum) {
awaitPolls(minimum, Collections.emptyList());
awaitPolls(minimum, List.of());
}
private void awaitPolls(int minimum) {
@ -1168,8 +1165,8 @@ public class ExactlyOnceWorkerSourceTaskTest {
private void expectPossibleTopicCreation() {
if (config.topicCreationEnable()) {
Set<String> created = Collections.singleton(TOPIC);
Set<String> existing = Collections.emptySet();
Set<String> created = Set.of(TOPIC);
Set<String> existing = Set.of();
TopicAdmin.TopicCreationResponse creationResponse = new TopicAdmin.TopicCreationResponse(created, existing);
when(admin.createOrFindTopics(any())).thenReturn(creationResponse);
}

View File

@ -27,7 +27,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.List;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -49,7 +49,7 @@ public class InternalSinkRecordTest {
assertTrue(sinkRecord.headers().isEmpty());
SinkRecord newRecord = internalSinkRecord.newRecord(TOPIC, 0, null, null, null,
null, null, Collections.singletonList(mock(Header.class)));
null, null, List.of(mock(Header.class)));
assertEquals(1, newRecord.headers().size());
}

View File

@ -34,8 +34,6 @@ 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.HashMap;
import java.util.List;
import java.util.Map;
@ -72,7 +70,7 @@ public class MockLoggersTest {
Loggers loggers = new TestLoggers(root, a, b);
Map<String, LoggerLevel> expectedLevels = Collections.singletonMap(
Map<String, LoggerLevel> expectedLevels = Map.of(
"b",
new LoggerLevel(Level.INFO.toString(), null)
);
@ -137,7 +135,7 @@ public class MockLoggersTest {
TestLoggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel("a.b.c.p", Level.WARN.name());
assertEquals(Arrays.asList("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified);
assertEquals(List.of("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified);
assertEquals(Level.WARN.toString(), loggers.level("a.b.c.p").level());
assertEquals(Level.WARN, x.getLevel());
assertEquals(Level.WARN, y.getLevel());
@ -201,7 +199,7 @@ public class MockLoggersTest {
Loggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel(rootLoggerName, Level.DEBUG.name());
assertEquals(Arrays.asList("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", rootLoggerName), modified);
assertEquals(List.of("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", rootLoggerName), modified);
assertEquals(Level.DEBUG, p.getLevel());

View File

@ -40,7 +40,6 @@ import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import static java.util.Collections.singletonMap;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ -94,7 +93,7 @@ public class SourceTaskOffsetCommitterTest {
committer.schedule(taskId, task);
assertNotNull(taskWrapper.getValue());
assertEquals(singletonMap(taskId, commitFuture), committers);
assertEquals(Map.of(taskId, commitFuture), committers);
}
@Test

View File

@ -24,6 +24,7 @@ import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@ -37,9 +38,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class SubmittedRecordsTest {
private static final Map<String, Object> PARTITION1 = Collections.singletonMap("subreddit", "apachekafka");
private static final Map<String, Object> PARTITION2 = Collections.singletonMap("subreddit", "adifferentvalue");
private static final Map<String, Object> PARTITION3 = Collections.singletonMap("subreddit", "asdfqweoicus");
private static final Map<String, Object> PARTITION1 = Map.of("subreddit", "apachekafka");
private static final Map<String, Object> PARTITION2 = Map.of("subreddit", "adifferentvalue");
private static final Map<String, Object> PARTITION3 = Map.of("subreddit", "asdfqweoicus");
private AtomicInteger offset;
@ -68,22 +69,22 @@ public class SubmittedRecordsTest {
@Test
public void testNoCommittedRecords() {
for (int i = 0; i < 3; i++) {
for (Map<String, Object> partition : Arrays.asList(PARTITION1, PARTITION2, PARTITION3)) {
for (Map<String, Object> partition : List.of(PARTITION1, PARTITION2, PARTITION3)) {
submittedRecords.submit(partition, newOffset());
}
}
CommittableOffsets committableOffsets = submittedRecords.committableOffsets();
assertMetadata(committableOffsets, 0, 9, 3, 3, PARTITION1, PARTITION2, PARTITION3);
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
committableOffsets = submittedRecords.committableOffsets();
assertMetadata(committableOffsets, 0, 9, 3, 3, PARTITION1, PARTITION2, PARTITION3);
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
committableOffsets = submittedRecords.committableOffsets();
assertMetadata(committableOffsets, 0, 9, 3, 3, PARTITION1, PARTITION2, PARTITION3);
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
}
@Test
@ -94,7 +95,7 @@ public class SubmittedRecordsTest {
CommittableOffsets committableOffsets = submittedRecords.committableOffsets();
// Record has been submitted but not yet acked; cannot commit offsets for it yet
assertFalse(committableOffsets.isEmpty());
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 1, 1, 1, PARTITION1);
assertNoEmptyDeques();
@ -102,7 +103,7 @@ public class SubmittedRecordsTest {
committableOffsets = submittedRecords.committableOffsets();
// Record has been acked; can commit offsets for it
assertFalse(committableOffsets.isEmpty());
assertEquals(Collections.singletonMap(PARTITION1, offset), committableOffsets.offsets());
assertEquals(Map.of(PARTITION1, offset), committableOffsets.offsets());
assertMetadataNoPending(committableOffsets, 1);
// Everything has been ack'd and consumed; make sure that it's been cleaned up to avoid memory leaks
@ -110,7 +111,7 @@ public class SubmittedRecordsTest {
committableOffsets = submittedRecords.committableOffsets();
// Old offsets should be wiped
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertTrue(committableOffsets.isEmpty());
}
@ -128,27 +129,27 @@ public class SubmittedRecordsTest {
CommittableOffsets committableOffsets = submittedRecords.committableOffsets();
// No records ack'd yet; can't commit any offsets
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 4, 2, 2, PARTITION1, PARTITION2);
assertNoEmptyDeques();
partition1Record2.ack();
committableOffsets = submittedRecords.committableOffsets();
// One record has been ack'd, but a record that comes before it and corresponds to the same source partition hasn't been
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 4, 2, 2, PARTITION1, PARTITION2);
assertNoEmptyDeques();
partition2Record1.ack();
committableOffsets = submittedRecords.committableOffsets();
// We can commit the first offset for the second partition
assertEquals(Collections.singletonMap(PARTITION2, partition2Offset1), committableOffsets.offsets());
assertEquals(Map.of(PARTITION2, partition2Offset1), committableOffsets.offsets());
assertMetadata(committableOffsets, 1, 3, 2, 2, PARTITION1);
assertNoEmptyDeques();
committableOffsets = submittedRecords.committableOffsets();
// No new offsets to commit
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 3, 2, 2, PARTITION1);
assertNoEmptyDeques();
@ -176,7 +177,7 @@ public class SubmittedRecordsTest {
SubmittedRecord submittedRecord = submittedRecords.submit(PARTITION1, newOffset());
CommittableOffsets committableOffsets = submittedRecords.committableOffsets();
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 1, 1, 1, PARTITION1);
assertTrue(submittedRecord.drop(), "First attempt to remove record from submitted queue should succeed");
@ -208,7 +209,7 @@ public class SubmittedRecordsTest {
committableOffsets = submittedRecords.committableOffsets();
// Even if SubmittedRecords::remove is broken, we haven't ack'd anything yet, so there should be no committable offsets
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 1, 1, 1, PARTITION2);
assertNoEmptyDeques();
// The only record for this partition has been removed; we shouldn't be tracking a deque for it anymore
@ -217,14 +218,14 @@ public class SubmittedRecordsTest {
recordToRemove.ack();
committableOffsets = submittedRecords.committableOffsets();
// Even though the record has somehow been acknowledged, it should not be counted when collecting committable offsets
assertEquals(Collections.emptyMap(), committableOffsets.offsets());
assertEquals(Map.of(), committableOffsets.offsets());
assertMetadata(committableOffsets, 0, 1, 1, 1, PARTITION2);
assertNoEmptyDeques();
lastSubmittedRecord.ack();
committableOffsets = submittedRecords.committableOffsets();
// Now that the last-submitted record has been ack'd, we should be able to commit its offset
assertEquals(Collections.singletonMap(PARTITION2, partition2Offset), committableOffsets.offsets());
assertEquals(Map.of(PARTITION2, partition2Offset), committableOffsets.offsets());
assertMetadata(committableOffsets, 1, 0, 0, 0, (Map<String, Object>) null);
assertFalse(committableOffsets.hasPending());
@ -338,7 +339,7 @@ public class SubmittedRecordsTest {
}
private void assertNoRemainingDeques() {
assertEquals(Collections.emptyMap(), submittedRecords.records, "Internal records map should be completely empty");
assertEquals(Map.of(), submittedRecords.records, "Internal records map should be completely empty");
}
@SafeVarargs
@ -355,7 +356,7 @@ public class SubmittedRecordsTest {
}
private Map<String, Object> newOffset() {
return Collections.singletonMap("timestamp", offset.getAndIncrement());
return Map.of("timestamp", offset.getAndIncrement());
}
private void assertMetadataNoPending(CommittableOffsets committableOffsets, int committableMessages) {

View File

@ -26,7 +26,8 @@ import org.junit.jupiter.api.Test;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import static java.util.Collections.singletonMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.Mockito.mock;
@ -36,8 +37,8 @@ import static org.mockito.Mockito.when;
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class TransformationStageTest {
private final SourceRecord initial = new SourceRecord(singletonMap("initial", 1), null, null, null, null);
private final SourceRecord transformed = new SourceRecord(singletonMap("transformed", 2), null, null, null, null);
private final SourceRecord initial = new SourceRecord(Map.of("initial", 1), null, null, null, null);
private final SourceRecord transformed = new SourceRecord(Map.of("transformed", 2), null, null, null, null);
@Test
public void apply() throws Exception {

View File

@ -27,7 +27,6 @@ import org.junit.jupiter.api.Test;
import org.mockito.MockedStatic;
import org.mockito.internal.stubbing.answers.CallsRealMethods;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -59,7 +58,7 @@ public class WorkerConfigTest {
public void testLookupKafkaClusterId() {
final Node broker1 = new Node(0, "dummyHost-1", 1234);
final Node broker2 = new Node(1, "dummyHost-2", 1234);
List<Node> cluster = Arrays.asList(broker1, broker2);
List<Node> cluster = List.of(broker1, broker2);
MockAdminClient adminClient = new MockAdminClient.Builder().
brokers(cluster).build();
assertEquals(MockAdminClient.DEFAULT_CLUSTER_ID, WorkerConfig.lookupKafkaClusterId(adminClient));
@ -69,7 +68,7 @@ public class WorkerConfigTest {
public void testLookupNullKafkaClusterId() {
final Node broker1 = new Node(0, "dummyHost-1", 1234);
final Node broker2 = new Node(1, "dummyHost-2", 1234);
List<Node> cluster = Arrays.asList(broker1, broker2);
List<Node> cluster = List.of(broker1, broker2);
MockAdminClient adminClient = new MockAdminClient.Builder().
brokers(cluster).clusterId(null).build();
assertNull(WorkerConfig.lookupKafkaClusterId(adminClient));
@ -79,7 +78,7 @@ public class WorkerConfigTest {
public void testLookupKafkaClusterIdTimeout() {
final Node broker1 = new Node(0, "dummyHost-1", 1234);
final Node broker2 = new Node(1, "dummyHost-2", 1234);
List<Node> cluster = Arrays.asList(broker1, broker2);
List<Node> cluster = List.of(broker1, broker2);
MockAdminClient adminClient = new MockAdminClient.Builder().
brokers(cluster).build();
adminClient.timeoutNextRequest(1);

View File

@ -27,7 +27,6 @@ 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.Map;
import java.util.Set;
@ -66,13 +65,13 @@ public class WorkerConfigTransformerTest {
@BeforeEach
public void setup() {
configTransformer = new WorkerConfigTransformer(worker, Collections.singletonMap("test", new TestConfigProvider()));
configTransformer = new WorkerConfigTransformer(worker, Map.of("test", new TestConfigProvider()));
}
@Test
public void testReplaceVariable() {
// Execution
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Collections.singletonMap(MY_KEY, "${test:testPath:testKey}"));
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Map.of(MY_KEY, "${test:testPath:testKey}"));
// Assertions
assertEquals(TEST_RESULT, result.get(MY_KEY));
@ -97,7 +96,7 @@ public class WorkerConfigTransformerTest {
when(herder.restartConnector(eq(1L), eq(MY_CONNECTOR), notNull())).thenReturn(requestId);
// Execution
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Collections.singletonMap(MY_KEY, "${test:testPath:testKeyWithTTL}"));
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Map.of(MY_KEY, "${test:testPath:testKeyWithTTL}"));
// Assertions
assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY));
@ -112,14 +111,14 @@ public class WorkerConfigTransformerTest {
when(herder.restartConnector(eq(10L), eq(MY_CONNECTOR), notNull())).thenReturn(requestId);
// Execution
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Collections.singletonMap(MY_KEY, "${test:testPath:testKeyWithTTL}"));
Map<String, String> result = configTransformer.transform(MY_CONNECTOR, Map.of(MY_KEY, "${test:testPath:testKeyWithTTL}"));
// Assertions
assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY));
verify(herder).restartConnector(eq(1L), eq(MY_CONNECTOR), notNull());
// Execution
result = configTransformer.transform(MY_CONNECTOR, Collections.singletonMap(MY_KEY, "${test:testPath:testKeyWithLongerTTL}"));
result = configTransformer.transform(MY_CONNECTOR, Map.of(MY_KEY, "${test:testPath:testKeyWithLongerTTL}"));
// Assertions
assertEquals(TEST_RESULT_WITH_LONGER_TTL, result.get(MY_KEY));
@ -147,14 +146,14 @@ public class WorkerConfigTransformerTest {
public ConfigData get(String path, Set<String> keys) {
if (path.equals(TEST_PATH)) {
if (keys.contains(TEST_KEY)) {
return new ConfigData(Collections.singletonMap(TEST_KEY, TEST_RESULT));
return new ConfigData(Map.of(TEST_KEY, TEST_RESULT));
} else if (keys.contains(TEST_KEY_WITH_TTL)) {
return new ConfigData(Collections.singletonMap(TEST_KEY_WITH_TTL, TEST_RESULT_WITH_TTL), 1L);
return new ConfigData(Map.of(TEST_KEY_WITH_TTL, TEST_RESULT_WITH_TTL), 1L);
} else if (keys.contains(TEST_KEY_WITH_LONGER_TTL)) {
return new ConfigData(Collections.singletonMap(TEST_KEY_WITH_LONGER_TTL, TEST_RESULT_WITH_LONGER_TTL), 10L);
return new ConfigData(Map.of(TEST_KEY_WITH_LONGER_TTL, TEST_RESULT_WITH_LONGER_TTL), 10L);
}
}
return new ConfigData(Collections.emptyMap());
return new ConfigData(Map.of());
}
@Override

View File

@ -74,9 +74,7 @@ import org.mockito.stubbing.OngoingStubbing;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
@ -91,8 +89,6 @@ import java.util.function.Supplier;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import static java.util.Arrays.asList;
import static java.util.Collections.singleton;
import static org.apache.kafka.connect.runtime.WorkerTestUtils.getTransformationChain;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -135,7 +131,7 @@ public class WorkerSinkTaskTest {
private static final TopicPartition TOPIC_PARTITION3 = new TopicPartition(TOPIC, PARTITION3);
private static final Set<TopicPartition> INITIAL_ASSIGNMENT =
new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
Set.of(TOPIC_PARTITION, TOPIC_PARTITION2);
private static final Map<String, String> TASK_PROPS = new HashMap<>();
@ -197,11 +193,11 @@ public class WorkerSinkTaskTest {
}
private void createTask(TargetState initialState, TransformationChain transformationChain, RetryWithToleranceOperator toleranceOperator) {
createTask(initialState, keyConverter, valueConverter, headerConverter, toleranceOperator, Collections::emptyList, transformationChain);
createTask(initialState, keyConverter, valueConverter, headerConverter, toleranceOperator, List::of, transformationChain);
}
private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) {
createTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noneOperator(), Collections::emptyList, transformationChain);
createTask(initialState, keyConverter, valueConverter, headerConverter, RetryWithToleranceOperatorTest.noneOperator(), List::of, transformationChain);
}
private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter,
@ -318,7 +314,7 @@ public class WorkerSinkTaskTest {
verify(consumer).wakeup();
// Offset commit as requested when pausing; No records returned by consumer.poll()
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
workerTask.iteration(); // now paused
time.sleep(30000L);
@ -340,7 +336,7 @@ public class WorkerSinkTaskTest {
// And unpause
verify(statusListener).onResume(taskId);
verify(consumer, times(2)).wakeup();
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(singleton(tp)));
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Set.of(tp)));
verify(sinkTask, times(4)).put(anyList());
}
@ -363,7 +359,7 @@ public class WorkerSinkTaskTest {
sinkTaskContext.getValue().requestCommit(); // Force an offset commit
// second iteration
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
workerTask.iteration();
verify(sinkTask, times(2)).put(anyList());
@ -441,7 +437,7 @@ public class WorkerSinkTaskTest {
time.sleep(30000L);
verify(sinkTask, times(3)).put(anyList());
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Collections.singleton(tp)));
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Set.of(tp)));
assertSinkMetricValue("sink-record-read-total", 1.0);
assertSinkMetricValue("sink-record-send-total", 1.0);
@ -488,9 +484,9 @@ public class WorkerSinkTaskTest {
when(consumer.assignment())
.thenReturn(INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT, INITIAL_ASSIGNMENT)
.thenReturn(newAssignment, newAssignment, newAssignment)
.thenReturn(Collections.singleton(TOPIC_PARTITION3),
Collections.singleton(TOPIC_PARTITION3),
Collections.singleton(TOPIC_PARTITION3));
.thenReturn(Set.of(TOPIC_PARTITION3),
Set.of(TOPIC_PARTITION3),
Set.of(TOPIC_PARTITION3));
INITIAL_ASSIGNMENT.forEach(tp -> when(consumer.position(tp)).thenReturn(FIRST_OFFSET));
when(consumer.position(TOPIC_PARTITION3)).thenReturn(FIRST_OFFSET);
@ -503,8 +499,8 @@ public class WorkerSinkTaskTest {
.thenAnswer(expectConsumerPoll(1))
// Empty consumer poll (all partitions are paused) with rebalance; one new partition is assigned
.thenAnswer(invocation -> {
rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet());
rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3));
rebalanceListener.getValue().onPartitionsRevoked(Set.of());
rebalanceListener.getValue().onPartitionsAssigned(Set.of(TOPIC_PARTITION3));
return ConsumerRecords.empty();
})
.thenAnswer(expectConsumerPoll(0))
@ -513,7 +509,7 @@ public class WorkerSinkTaskTest {
ConsumerRecord<byte[], byte[]> newRecord = new ConsumerRecord<>(TOPIC, PARTITION3, FIRST_OFFSET, RAW_KEY, RAW_VALUE);
rebalanceListener.getValue().onPartitionsRevoked(INITIAL_ASSIGNMENT);
rebalanceListener.getValue().onPartitionsAssigned(Collections.emptyList());
rebalanceListener.getValue().onPartitionsAssigned(List.of());
return new ConsumerRecords<>(Map.of(TOPIC_PARTITION3, List.of(newRecord)),
Map.of(TOPIC_PARTITION3, new OffsetAndMetadata(FIRST_OFFSET + 1, Optional.empty(), "")));
});
@ -534,7 +530,7 @@ public class WorkerSinkTaskTest {
verify(consumer).pause(INITIAL_ASSIGNMENT);
workerTask.iteration();
verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3));
verify(sinkTask).open(Set.of(TOPIC_PARTITION3));
// All partitions are re-paused in order to pause any newly-assigned partitions so that redelivery efforts can continue
verify(consumer).pause(newAssignment);
@ -543,13 +539,13 @@ public class WorkerSinkTaskTest {
final Map<TopicPartition, OffsetAndMetadata> offsets = INITIAL_ASSIGNMENT.stream()
.collect(Collectors.toMap(Function.identity(), tp -> new OffsetAndMetadata(FIRST_OFFSET)));
when(sinkTask.preCommit(offsets)).thenReturn(offsets);
newAssignment = Collections.singleton(TOPIC_PARTITION3);
newAssignment = Set.of(TOPIC_PARTITION3);
workerTask.iteration();
verify(sinkTask).close(INITIAL_ASSIGNMENT);
// All partitions are resumed, as all previously paused-for-redelivery partitions were revoked
newAssignment.forEach(tp -> verify(consumer).resume(Collections.singleton(tp)));
newAssignment.forEach(tp -> verify(consumer).resume(Set.of(tp)));
}
@Test
@ -636,8 +632,8 @@ public class WorkerSinkTaskTest {
when(consumer.assignment())
.thenReturn(INITIAL_ASSIGNMENT)
.thenReturn(INITIAL_ASSIGNMENT)
.thenReturn(Collections.singleton(TOPIC_PARTITION2))
.thenReturn(Collections.singleton(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(INITIAL_ASSIGNMENT)
@ -656,18 +652,18 @@ public class WorkerSinkTaskTest {
return ConsumerRecords.empty();
})
.thenAnswer((Answer<ConsumerRecords<byte[], byte[]>>) invocation -> {
rebalanceListener.getValue().onPartitionsRevoked(singleton(TOPIC_PARTITION));
rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet());
rebalanceListener.getValue().onPartitionsRevoked(Set.of(TOPIC_PARTITION));
rebalanceListener.getValue().onPartitionsAssigned(Set.of());
return ConsumerRecords.empty();
})
.thenAnswer((Answer<ConsumerRecords<byte[], byte[]>>) invocation -> {
rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet());
rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION3));
rebalanceListener.getValue().onPartitionsRevoked(Set.of());
rebalanceListener.getValue().onPartitionsAssigned(Set.of(TOPIC_PARTITION3));
return ConsumerRecords.empty();
})
.thenAnswer((Answer<ConsumerRecords<byte[], byte[]>>) invocation -> {
rebalanceListener.getValue().onPartitionsLost(singleton(TOPIC_PARTITION3));
rebalanceListener.getValue().onPartitionsAssigned(singleton(TOPIC_PARTITION));
rebalanceListener.getValue().onPartitionsLost(Set.of(TOPIC_PARTITION3));
rebalanceListener.getValue().onPartitionsAssigned(Set.of(TOPIC_PARTITION));
return ConsumerRecords.empty();
});
@ -683,19 +679,19 @@ public class WorkerSinkTaskTest {
// Second iteration--second call to poll, partial consumer revocation
workerTask.iteration();
verify(sinkTask).close(singleton(TOPIC_PARTITION));
verify(sinkTask, times(2)).put(Collections.emptyList());
verify(sinkTask).close(Set.of(TOPIC_PARTITION));
verify(sinkTask, times(2)).put(List.of());
// Third iteration--third call to poll, partial consumer assignment
workerTask.iteration();
verify(sinkTask).open(singleton(TOPIC_PARTITION3));
verify(sinkTask, times(3)).put(Collections.emptyList());
verify(sinkTask).open(Set.of(TOPIC_PARTITION3));
verify(sinkTask, times(3)).put(List.of());
// Fourth iteration--fourth call to poll, one partition lost; can't commit offsets for it, one new partition assigned
workerTask.iteration();
verify(sinkTask).close(singleton(TOPIC_PARTITION3));
verify(sinkTask).open(singleton(TOPIC_PARTITION));
verify(sinkTask, times(4)).put(Collections.emptyList());
verify(sinkTask).close(Set.of(TOPIC_PARTITION3));
verify(sinkTask).open(Set.of(TOPIC_PARTITION));
verify(sinkTask, times(4)).put(List.of());
}
@Test
@ -730,14 +726,14 @@ public class WorkerSinkTaskTest {
.thenAnswer(expectConsumerPoll(1))
// Third poll; assignment changes to [TP2]
.thenAnswer(invocation -> {
rebalanceListener.getValue().onPartitionsRevoked(Collections.singleton(TOPIC_PARTITION));
rebalanceListener.getValue().onPartitionsAssigned(Collections.emptySet());
rebalanceListener.getValue().onPartitionsRevoked(Set.of(TOPIC_PARTITION));
rebalanceListener.getValue().onPartitionsAssigned(Set.of());
return ConsumerRecords.empty();
})
// Fourth poll; assignment changes to [TP2, TP3]
.thenAnswer(invocation -> {
rebalanceListener.getValue().onPartitionsRevoked(Collections.emptySet());
rebalanceListener.getValue().onPartitionsAssigned(Collections.singleton(TOPIC_PARTITION3));
rebalanceListener.getValue().onPartitionsRevoked(Set.of());
rebalanceListener.getValue().onPartitionsAssigned(Set.of(TOPIC_PARTITION3));
return ConsumerRecords.empty();
})
// Fifth poll; an offset commit takes place
@ -756,13 +752,13 @@ public class WorkerSinkTaskTest {
doNothing().when(consumer).commitSync(offsets);
workerTask.iteration();
verify(sinkTask).close(Collections.singleton(TOPIC_PARTITION));
verify(sinkTask, times(2)).put(Collections.emptyList());
verify(sinkTask).close(Set.of(TOPIC_PARTITION));
verify(sinkTask, times(2)).put(List.of());
// Fourth iteration--fourth call to poll, partial consumer assignment
workerTask.iteration();
verify(sinkTask).open(Collections.singleton(TOPIC_PARTITION3));
verify(sinkTask).open(Set.of(TOPIC_PARTITION3));
final Map<TopicPartition, OffsetAndMetadata> workerCurrentOffsets = new HashMap<>();
workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET));
@ -819,7 +815,7 @@ public class WorkerSinkTaskTest {
verify(sinkTask).close(INITIAL_ASSIGNMENT);
verify(sinkTask, times(2)).open(INITIAL_ASSIGNMENT);
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Collections.singleton(tp)));
INITIAL_ASSIGNMENT.forEach(tp -> verify(consumer).resume(Set.of(tp)));
verify(statusListener).onResume(taskId);
@ -905,7 +901,7 @@ public class WorkerSinkTaskTest {
@Test
public void testSkipsFailedRetriableExceptionFromConvert() {
createTask(initialState, keyConverter, valueConverter, headerConverter,
RetryWithToleranceOperatorTest.allOperator(), Collections::emptyList, transformationChain);
RetryWithToleranceOperatorTest.allOperator(), List::of, transformationChain);
workerTask.initialize(TASK_CONFIG);
workerTask.initializeAndStart();
@ -923,7 +919,7 @@ public class WorkerSinkTaskTest {
workerTask.iteration();
workerTask.execute();
verify(sinkTask, times(3)).put(Collections.emptyList());
verify(sinkTask, times(3)).put(List.of());
}
@Test
@ -974,7 +970,7 @@ public class WorkerSinkTaskTest {
workerTask.iteration();
workerTask.execute();
verify(sinkTask, times(3)).put(Collections.emptyList());
verify(sinkTask, times(3)).put(List.of());
}
@Test
@ -1273,7 +1269,7 @@ public class WorkerSinkTaskTest {
.when(sinkTask).put(anyList());
Throwable closeException = new RuntimeException();
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
// Throw another exception while closing the task's assignment
doThrow(closeException).when(sinkTask).close(any(Collection.class));
@ -1310,7 +1306,7 @@ public class WorkerSinkTaskTest {
.doThrow(putException)
.when(sinkTask).put(anyList());
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
// Throw another exception while closing the task's assignment
doThrow(closeException).when(sinkTask).close(any(Collection.class));
@ -1390,7 +1386,7 @@ public class WorkerSinkTaskTest {
workerCurrentOffsets.put(TOPIC_PARTITION2, new OffsetAndMetadata(FIRST_OFFSET));
final List<TopicPartition> originalPartitions = new ArrayList<>(INITIAL_ASSIGNMENT);
final List<TopicPartition> rebalancedPartitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
final List<TopicPartition> rebalancedPartitions = List.of(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
final Map<TopicPartition, OffsetAndMetadata> rebalanceOffsets = new HashMap<>();
rebalanceOffsets.put(TOPIC_PARTITION, workerCurrentOffsets.get(TOPIC_PARTITION));
rebalanceOffsets.put(TOPIC_PARTITION2, workerCurrentOffsets.get(TOPIC_PARTITION2));
@ -1532,7 +1528,10 @@ public class WorkerSinkTaskTest {
assertEquals(rebalanceOffsets, workerTask.lastCommittedOffsets());
// onPartitionsRevoked
verify(sinkTask).close(new ArrayList<>(workerCurrentOffsets.keySet()));
ArgumentCaptor<Collection<TopicPartition>> closeCaptor = ArgumentCaptor.forClass(Collection.class);
verify(sinkTask).close(closeCaptor.capture());
Collection<TopicPartition> actualClosePartitions = closeCaptor.getValue();
assertEquals(workerCurrentOffsets.keySet(), new HashSet<>(actualClosePartitions));
verify(consumer).commitSync(anyMap());
// onPartitionsAssigned - step 2
@ -1816,7 +1815,7 @@ public class WorkerSinkTaskTest {
expectPollInitialAssignment()
.thenAnswer((Answer<ConsumerRecords<byte[], byte[]>>) invocation -> {
List<ConsumerRecord<byte[], byte[]>> records = Arrays.asList(
List<ConsumerRecord<byte[], byte[]>> records = List.of(
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 1, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE,
0, 0, keyA.getBytes(), valueA.getBytes(encodingA), headersA, Optional.empty()),
new ConsumerRecord<>(TOPIC, PARTITION, FIRST_OFFSET + recordsReturnedTp1 + 2, RecordBatch.NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE,
@ -1880,7 +1879,7 @@ public class WorkerSinkTaskTest {
createTask(taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, metrics,
keyConverter, valueConverter, errorHandlingMetrics, headerConverter,
transformationChain, mockConsumer, pluginLoader, time,
RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore, Collections::emptyList);
RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore, List::of);
mockConsumer.updateBeginningOffsets(
new HashMap<>() {{
put(TOPIC_PARTITION, 0L);
@ -1893,7 +1892,7 @@ public class WorkerSinkTaskTest {
mockConsumer.rebalance(INITIAL_ASSIGNMENT);
assertSinkMetricValue("partition-count", 2);
// Revoked "TOPIC_PARTITION" and second re-balance with "TOPIC_PARTITION2"
mockConsumer.rebalance(Collections.singleton(TOPIC_PARTITION2));
mockConsumer.rebalance(Set.of(TOPIC_PARTITION2));
assertSinkMetricValue("partition-count", 1);
// Closing the Worker Sink Task which will update the partition count as 0.
workerTask.close();
@ -1901,12 +1900,12 @@ public class WorkerSinkTaskTest {
}
private void expectRebalanceRevocationError(RuntimeException e) {
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
doThrow(e).when(sinkTask).close(INITIAL_ASSIGNMENT);
}
private void expectRebalanceAssignmentError(RuntimeException e) {
when(sinkTask.preCommit(anyMap())).thenReturn(Collections.emptyMap());
when(sinkTask.preCommit(anyMap())).thenReturn(Map.of());
when(consumer.position(TOPIC_PARTITION)).thenReturn(FIRST_OFFSET);
when(consumer.position(TOPIC_PARTITION2)).thenReturn(FIRST_OFFSET);
@ -1914,7 +1913,7 @@ public class WorkerSinkTaskTest {
}
private void verifyInitializeTask() {
verify(consumer).subscribe(eq(Collections.singletonList(TOPIC)), rebalanceListener.capture());
verify(consumer).subscribe(eq(List.of(TOPIC)), rebalanceListener.capture());
verify(sinkTask).initialize(sinkTaskContext.capture());
verify(sinkTask).start(TASK_PROPS);
}
@ -1934,7 +1933,7 @@ public class WorkerSinkTaskTest {
private void verifyPollInitialAssignment() {
verify(sinkTask).open(INITIAL_ASSIGNMENT);
verify(consumer, atLeastOnce()).assignment();
verify(sinkTask).put(Collections.emptyList());
verify(sinkTask).put(List.of());
}
private Answer<ConsumerRecords<byte[], byte[]>> expectConsumerPoll(final int numMessages) {

View File

@ -61,9 +61,7 @@ import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -71,7 +69,6 @@ import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import static java.util.Collections.singletonList;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ -182,7 +179,7 @@ public class WorkerSinkTaskThreadedTest {
taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverterPlugin,
valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain,
consumer, pluginLoader, time, RetryWithToleranceOperatorTest.noneOperator(), null, statusBackingStore,
Collections::emptyList, null, TestPlugins.noOpLoaderSwap());
List::of, null, TestPlugins.noOpLoaderSwap());
recordsReturned = 0;
}
@ -460,11 +457,11 @@ public class WorkerSinkTaskThreadedTest {
return null;
}).when(sinkTask).put(any(Collection.class));
doThrow(new IllegalStateException("unassigned topic partition")).when(consumer).pause(singletonList(UNASSIGNED_TOPIC_PARTITION));
doAnswer(invocation -> null).when(consumer).pause(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
doThrow(new IllegalStateException("unassigned topic partition")).when(consumer).pause(List.of(UNASSIGNED_TOPIC_PARTITION));
doAnswer(invocation -> null).when(consumer).pause(List.of(TOPIC_PARTITION, TOPIC_PARTITION2));
doThrow(new IllegalStateException("unassigned topic partition")).when(consumer).resume(singletonList(UNASSIGNED_TOPIC_PARTITION));
doAnswer(invocation -> null).when(consumer).resume(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
doThrow(new IllegalStateException("unassigned topic partition")).when(consumer).resume(List.of(UNASSIGNED_TOPIC_PARTITION));
doAnswer(invocation -> null).when(consumer).resume(List.of(TOPIC_PARTITION, TOPIC_PARTITION2));
workerTask.initialize(TASK_CONFIG);
workerTask.initializeAndStart();
@ -481,8 +478,8 @@ public class WorkerSinkTaskThreadedTest {
verifyStopTask();
verifyTaskGetTopic(3);
verify(consumer, atLeastOnce()).pause(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
verify(consumer, atLeastOnce()).resume(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2));
verify(consumer, atLeastOnce()).pause(List.of(TOPIC_PARTITION, TOPIC_PARTITION2));
verify(consumer, atLeastOnce()).resume(List.of(TOPIC_PARTITION, TOPIC_PARTITION2));
}
@Test
@ -557,7 +554,7 @@ public class WorkerSinkTaskThreadedTest {
}
private void verifyInitializeTask() {
verify(consumer).subscribe(eq(singletonList(TOPIC)), rebalanceListener.capture());
verify(consumer).subscribe(eq(List.of(TOPIC)), rebalanceListener.capture());
verify(sinkTask).initialize(sinkTaskContext.capture());
verify(sinkTask).start(TASK_PROPS);
}
@ -570,7 +567,7 @@ public class WorkerSinkTaskThreadedTest {
private void verifyInitialAssignment() {
verify(sinkTask).open(INITIAL_ASSIGNMENT);
verify(sinkTask).put(Collections.emptyList());
verify(sinkTask).put(List.of());
}
private void verifyStopTask() {
@ -614,7 +611,7 @@ public class WorkerSinkTaskThreadedTest {
@SuppressWarnings("SameParameterValue")
private void expectRebalanceDuringPoll(long startOffset) {
final List<TopicPartition> partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
final List<TopicPartition> partitions = List.of(TOPIC_PARTITION, TOPIC_PARTITION2, TOPIC_PARTITION3);
final Map<TopicPartition, Long> offsets = new HashMap<>();
offsets.put(TOPIC_PARTITION, startOffset);

View File

@ -71,8 +71,6 @@ import org.mockito.stubbing.Answer;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedList;
import java.util.List;
@ -128,8 +126,8 @@ public class WorkerSourceTaskTest {
public static final String POLL_TIMEOUT_MSG = "Timeout waiting for poll";
private static final String TOPIC = "topic";
private static final Map<String, Object> PARTITION = Collections.singletonMap("key", "partition".getBytes());
private static final Map<String, Object> OFFSET = Collections.singletonMap("key", 12);
private static final Map<String, Object> PARTITION = Map.of("key", "partition".getBytes());
private static final Map<String, Object> OFFSET = Map.of("key", 12);
// Connect-format data
private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA;
@ -185,7 +183,7 @@ public class WorkerSourceTaskTest {
private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS);
private static final List<SourceRecord> RECORDS = Collections.singletonList(
private static final List<SourceRecord> RECORDS = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)
);
@ -255,7 +253,7 @@ public class WorkerSourceTaskTest {
workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin,
transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig),
offsetReader, offsetWriter, offsetStore, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM,
retryWithToleranceOperator, statusBackingStore, Runnable::run, Collections::emptyList, null, TestPlugins.noOpLoaderSwap());
retryWithToleranceOperator, statusBackingStore, Runnable::run, List::of, null, TestPlugins.noOpLoaderSwap());
}
@ParameterizedTest
@ -504,7 +502,7 @@ public class WorkerSourceTaskTest {
final CountDownLatch pollLatch = expectPolls(1);
expectTopicCreation(TOPIC);
expectBeginFlush(Arrays.asList(true, false).iterator()::next);
expectBeginFlush(List.of(true, false).iterator()::next);
expectOffsetFlush(true, true);
workerTask.initialize(TASK_CONFIG);
@ -591,9 +589,9 @@ public class WorkerSourceTaskTest {
.thenAnswer(producerSendAnswer(true));
// Try to send 3, make first pass, second fail. Should save last two
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
workerTask.sendRecords();
assertEquals(Arrays.asList(record2, record3), workerTask.toSend);
assertEquals(List.of(record2, record3), workerTask.toSend);
// Next they all succeed
workerTask.sendRecords();
@ -613,7 +611,7 @@ public class WorkerSourceTaskTest {
expectSendRecordProducerCallbackFail();
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, () -> workerTask.sendRecords());
verify(transformationChain, times(2)).apply(any(), any(SourceRecord.class));
@ -636,7 +634,7 @@ public class WorkerSourceTaskTest {
when(producer.send(any(ProducerRecord.class), any(Callback.class)))
.thenThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC)));
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, () -> workerTask.sendRecords());
}
@ -660,7 +658,7 @@ public class WorkerSourceTaskTest {
.doNothing()
.when(sourceTask).commitRecord(any(SourceRecord.class), any(RecordMetadata.class));
workerTask.toSend = Arrays.asList(record1, record2, record3);
workerTask.toSend = List.of(record1, record2, record3);
workerTask.sendRecords();
assertNull(workerTask.toSend);
}
@ -673,7 +671,7 @@ public class WorkerSourceTaskTest {
expectTopicCreation(TOPIC);
//Use different offsets for each record, so we can verify all were committed
final Map<String, Object> offset2 = Collections.singletonMap("key", 13);
final Map<String, Object> offset2 = Map.of("key", 13);
// send two records
// record 1 will succeed
@ -690,7 +688,7 @@ public class WorkerSourceTaskTest {
.thenAnswer(producerSendAnswer(false));
//Send records and then commit offsets and verify both were committed and no exception
workerTask.toSend = Arrays.asList(record1, record2);
workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords();
workerTask.updateCommittableOffsets();
workerTask.commitOffsets();
@ -755,8 +753,8 @@ public class WorkerSourceTaskTest {
}
private TopicAdmin.TopicCreationResponse createdTopic(String topic) {
Set<String> created = Collections.singleton(topic);
Set<String> existing = Collections.emptySet();
Set<String> created = Set.of(topic);
Set<String> existing = Set.of();
return new TopicAdmin.TopicCreationResponse(created, existing);
}
@ -774,7 +772,7 @@ public class WorkerSourceTaskTest {
count.incrementAndGet();
latch.countDown();
Thread.sleep(10);
return Collections.emptyList();
return List.of();
});
return latch;
}
@ -896,7 +894,7 @@ public class WorkerSourceTaskTest {
private void expectOffsetFlush(Boolean... succeedList) throws Exception {
Future<Void> flushFuture = mock(Future.class);
when(offsetWriter.doFlush(any(org.apache.kafka.connect.util.Callback.class))).thenReturn(flushFuture);
LinkedList<Boolean> succeedQueue = new LinkedList<>(Arrays.asList(succeedList));
LinkedList<Boolean> succeedQueue = new LinkedList<>(List.of(succeedList));
doAnswer(invocationOnMock -> {
boolean succeed = succeedQueue.pop();
@ -993,7 +991,7 @@ public class WorkerSourceTaskTest {
private void expectTopicCreation(String topic) {
if (config.topicCreationEnable()) {
when(admin.describeTopics(topic)).thenReturn(Collections.emptyMap());
when(admin.describeTopics(topic)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(createdTopic(topic));
}
}
@ -1018,7 +1016,7 @@ public class WorkerSourceTaskTest {
committerAppender.setClassLogger(SourceTaskOffsetCommitter.class, org.apache.logging.log4j.Level.TRACE);
taskAppender.setClassLogger(WorkerSourceTask.class, org.apache.logging.log4j.Level.TRACE);
SourceTaskOffsetCommitter.commit(workerTask);
assertEquals(Collections.emptyList(), taskAppender.getMessages());
assertEquals(List.of(), taskAppender.getMessages());
List<String> committerMessages = committerAppender.getMessages();
assertEquals(1, committerMessages.size());

View File

@ -105,7 +105,6 @@ import org.mockito.invocation.InvocationOnMock;
import org.mockito.quality.Strictness;
import java.lang.management.ManagementFactory;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
@ -330,7 +329,7 @@ public class WorkerTest {
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy);
worker.start();
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
FutureCallback<TargetState> onFirstStart = new FutureCallback<>();
@ -338,7 +337,7 @@ public class WorkerTest {
// Wait for the connector to actually start
assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS));
assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames());
assertEquals(Set.of(CONNECTOR_ID), worker.connectorNames());
FutureCallback<TargetState> onSecondStart = new FutureCallback<>();
@ -352,7 +351,7 @@ public class WorkerTest {
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 0, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
// Nothing should be left, so this should effectively be a nop
worker.stop();
@ -373,7 +372,7 @@ public class WorkerTest {
private void mockFileConfigProvider() {
MockFileConfigProvider mockFileConfigProvider = new MockFileConfigProvider();
mockFileConfigProvider.configure(Collections.singletonMap("testId", mockFileProviderTestId));
mockFileConfigProvider.configure(Map.of("testId", mockFileProviderTestId));
when(plugins.newConfigProvider(any(AbstractConfig.class),
eq("config.providers.file"),
any(ClassLoaderUsage.class)))
@ -409,7 +408,7 @@ public class WorkerTest {
}
assertStartupStatistics(worker, 1, 1, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 1, 0, 0);
@ -440,19 +439,19 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
FutureCallback<TargetState> onStart = new FutureCallback<>();
worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart);
// Wait for the connector to actually start
assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS));
assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames());
assertEquals(Set.of(CONNECTOR_ID), worker.connectorNames());
assertStatistics(worker, 1, 0);
assertStartupStatistics(worker, 1, 0, 0, 0);
worker.stopAndAwaitConnector(CONNECTOR_ID);
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 0, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -486,17 +485,17 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
FutureCallback<TargetState> onStart = new FutureCallback<>();
worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onStart);
// Wait for the connector to actually start
assertEquals(TargetState.STARTED, onStart.get(1000, TimeUnit.MILLISECONDS));
assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames());
assertEquals(Set.of(CONNECTOR_ID), worker.connectorNames());
assertStatistics(worker, 1, 0);
worker.stopAndAwaitConnector(CONNECTOR_ID);
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -537,8 +536,8 @@ public class WorkerTest {
mockVersionedConnectorIsolation(connectorClass, null, sinkConnector);
mockExecutorRealSubmit(WorkerConnector.class);
Map<String, String> taskProps = Collections.singletonMap("foo", "bar");
when(sinkConnector.taskConfigs(2)).thenReturn(Arrays.asList(taskProps, taskProps));
Map<String, String> taskProps = Map.of("foo", "bar");
when(sinkConnector.taskConfigs(2)).thenReturn(List.of(taskProps, taskProps));
// Use doReturn().when() syntax due to when().thenReturn() not being able to return wildcard generic types
doReturn(TestSourceTask.class).when(sinkConnector).taskClass();
@ -552,13 +551,13 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
FutureCallback<TargetState> onFirstStart = new FutureCallback<>();
worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onFirstStart);
// Wait for the connector to actually start
assertEquals(TargetState.STARTED, onFirstStart.get(1000, TimeUnit.MILLISECONDS));
assertStatistics(worker, 1, 0);
assertEquals(Collections.singleton(CONNECTOR_ID), worker.connectorNames());
assertEquals(Set.of(CONNECTOR_ID), worker.connectorNames());
FutureCallback<TargetState> onSecondStart = new FutureCallback<>();
worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED, onSecondStart);
@ -581,7 +580,7 @@ public class WorkerTest {
worker.stopAndAwaitConnector(CONNECTOR_ID);
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 1, 0, 0, 0);
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -610,7 +609,7 @@ public class WorkerTest {
mockVersionedTaskHeaderConverterFromConnector(taskHeaderConverter);
mockExecutorFakeSubmit(WorkerTask.class);
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
noneConnectorClientConfigOverridePolicy, null);
@ -618,28 +617,28 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
Map<String, String> connectorConfigs = anyConnectorConfigMap();
ClusterConfigState configState = new ClusterConfigState(
0,
null,
Collections.singletonMap(CONNECTOR_ID, 1),
Collections.singletonMap(CONNECTOR_ID, connectorConfigs),
Collections.singletonMap(CONNECTOR_ID, TargetState.STARTED),
Collections.singletonMap(TASK_ID, origProps),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_ID, 1),
Map.of(CONNECTOR_ID, connectorConfigs),
Map.of(CONNECTOR_ID, TargetState.STARTED),
Map.of(TASK_ID, origProps),
Map.of(),
Map.of(),
Map.of(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Set.of(),
Set.of()
);
assertTrue(worker.startSourceTask(TASK_ID, configState, connectorConfigs, origProps, taskStatusListener, TargetState.STARTED));
assertStatistics(worker, 0, 1);
assertEquals(Collections.singleton(TASK_ID), worker.taskIds());
assertEquals(Set.of(TASK_ID), worker.taskIds());
worker.stopAndAwaitTask(TASK_ID);
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -666,7 +665,7 @@ public class WorkerTest {
mockVersionedTaskHeaderConverterFromConnector(taskHeaderConverter);
mockExecutorFakeSubmit(WorkerTask.class);
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
noneConnectorClientConfigOverridePolicy, null);
@ -674,7 +673,7 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
Map<String, String> connectorConfigs = anyConnectorConfigMap();
connectorConfigs.put(TOPICS_CONFIG, "t1");
connectorConfigs.put(CONNECTOR_CLASS_CONFIG, SampleSinkConnector.class.getName());
@ -682,22 +681,22 @@ public class WorkerTest {
ClusterConfigState configState = new ClusterConfigState(
0,
null,
Collections.singletonMap(CONNECTOR_ID, 1),
Collections.singletonMap(CONNECTOR_ID, connectorConfigs),
Collections.singletonMap(CONNECTOR_ID, TargetState.STARTED),
Collections.singletonMap(TASK_ID, origProps),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_ID, 1),
Map.of(CONNECTOR_ID, connectorConfigs),
Map.of(CONNECTOR_ID, TargetState.STARTED),
Map.of(TASK_ID, origProps),
Map.of(),
Map.of(),
Map.of(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Set.of(),
Set.of()
);
assertTrue(worker.startSinkTask(TASK_ID, configState, connectorConfigs, origProps, taskStatusListener, TargetState.STARTED));
assertStatistics(worker, 0, 1);
assertEquals(Collections.singleton(TASK_ID), worker.taskIds());
assertEquals(Set.of(TASK_ID), worker.taskIds());
worker.stopAndAwaitTask(TASK_ID);
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -741,7 +740,7 @@ public class WorkerTest {
Runnable preProducer = mock(Runnable.class);
Runnable postProducer = mock(Runnable.class);
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
noneConnectorClientConfigOverridePolicy, null);
@ -749,29 +748,29 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
Map<String, String> connectorConfigs = anyConnectorConfigMap();
ClusterConfigState configState = new ClusterConfigState(
0,
null,
Collections.singletonMap(CONNECTOR_ID, 1),
Collections.singletonMap(CONNECTOR_ID, connectorConfigs),
Collections.singletonMap(CONNECTOR_ID, TargetState.STARTED),
Collections.singletonMap(TASK_ID, origProps),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_ID, 1),
Map.of(CONNECTOR_ID, connectorConfigs),
Map.of(CONNECTOR_ID, TargetState.STARTED),
Map.of(TASK_ID, origProps),
Map.of(),
Map.of(),
Map.of(CONNECTOR_ID, new AppliedConnectorConfig(connectorConfigs)),
Set.of(),
Set.of()
);
assertTrue(worker.startExactlyOnceSourceTask(TASK_ID, configState, connectorConfigs, origProps, taskStatusListener, TargetState.STARTED, preProducer, postProducer));
assertStatistics(worker, 0, 1);
assertEquals(Collections.singleton(TASK_ID), worker.taskIds());
assertEquals(Set.of(TASK_ID), worker.taskIds());
worker.stopAndAwaitTask(TASK_ID);
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -792,7 +791,7 @@ public class WorkerTest {
mockStorage();
mockFileConfigProvider();
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
TaskConfig taskConfig = new TaskConfig(origProps);
@ -827,7 +826,7 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 0, 0, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
worker.startSourceTask(
TASK_ID,
ClusterConfigState.EMPTY,
@ -905,7 +904,7 @@ public class WorkerTest {
mockInternalConverters();
mockFileConfigProvider();
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath");
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath");
mockKafkaClusterId();
mockGenericIsolation();
@ -922,7 +921,7 @@ public class WorkerTest {
assertStatistics(worker, 0, 0);
assertStartupStatistics(worker, 0, 0, 1, 1);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
verify(taskStatusListener).onFailure(eq(TASK_ID), any(ConfigException.class));
verifyKafkaClusterId();
@ -948,7 +947,7 @@ public class WorkerTest {
mockVersionedTaskHeaderConverterFromWorker(taskHeaderConverter);
mockExecutorFakeSubmit(WorkerTask.class);
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
TaskConfig taskConfig = new TaskConfig(origProps);
@ -990,7 +989,7 @@ public class WorkerTest {
mockStorage();
mockFileConfigProvider();
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName());
TaskConfig taskConfig = new TaskConfig(origProps);
mockKafkaClusterId();
@ -1009,16 +1008,16 @@ public class WorkerTest {
worker.herder = herder;
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
Map<String, String> connProps = anyConnectorConfigMap();
connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName());
connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, SampleConverterWithHeaders.class.getName());
worker.startSourceTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED);
assertStatistics(worker, 0, 1);
assertEquals(Collections.singleton(TASK_ID), worker.taskIds());
assertEquals(Set.of(TASK_ID), worker.taskIds());
worker.stopAndAwaitTask(TASK_ID);
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
// Nothing should be left, so this should effectively be a nop
worker.stop();
assertStatistics(worker, 0, 0);
@ -1206,7 +1205,7 @@ public class WorkerTest {
props.put("consumer.bootstrap.servers", "localhost:4761");
WorkerConfig configWithOverrides = new StandaloneConfig(props);
Map<String, Object> connConfig = Collections.singletonMap("metadata.max.age.ms", "10000");
Map<String, Object> connConfig = Map.of("metadata.max.age.ms", "10000");
Map<String, String> expectedConfigs = new HashMap<>(workerProps);
expectedConfigs.remove(AbstractConfig.CONFIG_PROVIDERS_CONFIG);
expectedConfigs.put("bootstrap.servers", "localhost:9092");
@ -1231,7 +1230,7 @@ public class WorkerTest {
props.put("admin.client.id", "testid");
props.put("admin.metadata.max.age.ms", "5000");
WorkerConfig configWithOverrides = new StandaloneConfig(props);
Map<String, Object> connConfig = Collections.singletonMap("metadata.max.age.ms", "10000");
Map<String, Object> connConfig = Map.of("metadata.max.age.ms", "10000");
when(connectorConfig.originalsWithPrefix(CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)).thenReturn(connConfig);
@ -1817,7 +1816,7 @@ public class WorkerTest {
noneConnectorClientConfigOverridePolicy, null);
worker.start();
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
worker.stop();
verifyKafkaClusterId();
verify(executorService, times(1)).shutdown();
@ -1839,7 +1838,7 @@ public class WorkerTest {
noneConnectorClientConfigOverridePolicy, null);
worker.start();
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
worker.stop();
verifyKafkaClusterId();
verify(executorService, times(1)).shutdown();
@ -1862,7 +1861,7 @@ public class WorkerTest {
noneConnectorClientConfigOverridePolicy, null);
worker.start();
assertEquals(Collections.emptySet(), worker.connectorNames());
assertEquals(Set.of(), worker.connectorNames());
worker.stop();
// Clear the interrupted status so that the test infrastructure doesn't hit an unexpected interrupt.
assertTrue(Thread.interrupted());
@ -1931,7 +1930,7 @@ public class WorkerTest {
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
allConnectorClientConfigOverridePolicy, config -> admin);
worker.start();
mockAdminListConsumerGroupOffsets(admin, Collections.singletonMap(new TopicPartition("test-topic", 0), new OffsetAndMetadata(10)), null);
mockAdminListConsumerGroupOffsets(admin, Map.of(new TopicPartition("test-topic", 0), new OffsetAndMetadata(10)), null);
FutureCallback<ConnectorOffsets> cb = new FutureCallback<>();
worker.sinkConnectorOffsets(CONNECTOR_ID, sinkConnector, connectorProps, cb);
@ -2038,11 +2037,11 @@ public class WorkerTest {
worker.start();
Set<Map<String, Object>> connectorPartitions =
Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue"));
Set.of(Map.of("partitionKey", "partitionValue"));
Map<Map<String, Object>, Map<String, Object>> partitionOffsets = Collections.singletonMap(
Collections.singletonMap("partitionKey", "partitionValue"),
Collections.singletonMap("offsetKey", "offsetValue")
Map<Map<String, Object>, Map<String, Object>> partitionOffsets = Map.of(
Map.of("partitionKey", "partitionValue"),
Map.of("offsetKey", "offsetValue")
);
when(offsetStore.connectorPartitions(CONNECTOR_ID)).thenReturn(connectorPartitions);
@ -2112,7 +2111,7 @@ public class WorkerTest {
FutureCallback<Message> cb = new FutureCallback<>();
worker.modifyConnectorOffsets(CONNECTOR_ID, connectorProps,
Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
Map.of(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
cb);
ExecutionException e = assertThrows(ExecutionException.class, () -> cb.get(1000, TimeUnit.MILLISECONDS));
@ -2142,8 +2141,8 @@ public class WorkerTest {
OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
partitionOffsets.put(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue"));
partitionOffsets.put(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"));
when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
invocation.getArgument(0, Callback.class).onCompletion(null, null);
@ -2181,8 +2180,8 @@ public class WorkerTest {
OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = new HashMap<>();
partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue"));
partitionOffsets.put(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"));
partitionOffsets.put(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue"));
partitionOffsets.put(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"));
when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
invocation.getArgument(0, Callback.class).onCompletion(new RuntimeException("Test exception"), null);
@ -2207,9 +2206,9 @@ public class WorkerTest {
@ValueSource(booleans = {true, false})
public void testNormalizeSourceConnectorOffsets(boolean enableTopicCreation) {
setup(enableTopicCreation);
Map<Map<String, ?>, Map<String, ?>> offsets = Collections.singletonMap(
Collections.singletonMap("filename", "/path/to/filename"),
Collections.singletonMap("position", 20)
Map<Map<String, ?>, Map<String, ?>> offsets = Map.of(
Map.of("filename", "/path/to/filename"),
Map.of("position", 20)
);
assertInstanceOf(Integer.class, offsets.values().iterator().next().get("position"));
@ -2237,11 +2236,11 @@ public class WorkerTest {
Map<String, String> partition1 = new HashMap<>();
partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 500));
partitionOffsets.put(partition1, Map.of(SinkUtils.KAFKA_OFFSET_KEY, 500));
Map<String, String> partition2 = new HashMap<>();
partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
partitionOffsets.put(partition2, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 100));
partitionOffsets.put(partition2, Map.of(SinkUtils.KAFKA_OFFSET_KEY, 100));
// A null value for deleteOffsetsSetCapture indicates that we don't expect any call to Admin::deleteConsumerGroupOffsets
alterOffsetsSinkConnector(partitionOffsets, alterOffsetsMapCapture, null);
@ -2290,7 +2289,7 @@ public class WorkerTest {
Map<String, String> partition1 = new HashMap<>();
partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
partitionOffsets.put(partition1, Map.of(SinkUtils.KAFKA_OFFSET_KEY, "100"));
Map<String, String> partition2 = new HashMap<>();
partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
@ -2301,7 +2300,7 @@ public class WorkerTest {
assertEquals(1, alterOffsetsMapCapture.getValue().size());
assertEquals(100, alterOffsetsMapCapture.getValue().get(new TopicPartition("test_topic", 10)).offset());
Set<TopicPartition> expectedTopicPartitionsForOffsetDelete = Collections.singleton(new TopicPartition("test_topic", 20));
Set<TopicPartition> expectedTopicPartitionsForOffsetDelete = Set.of(new TopicPartition("test_topic", 20));
assertEquals(expectedTopicPartitionsForOffsetDelete, deleteOffsetsSetCapture.getValue());
}
@ -2375,8 +2374,8 @@ public class WorkerTest {
Map<String, String> partition1 = new HashMap<>();
partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = Collections.singletonMap(partition1,
Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = Map.of(partition1,
Map.of(SinkUtils.KAFKA_OFFSET_KEY, "100"));
FutureCallback<Message> cb = new FutureCallback<>();
worker.modifySinkConnectorOffsets(CONNECTOR_ID, sinkConnector, connectorProps, partitionOffsets,
@ -2423,7 +2422,7 @@ public class WorkerTest {
Map<String, String> partition1 = new HashMap<>();
partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
partitionOffsets.put(partition1, Map.of(SinkUtils.KAFKA_OFFSET_KEY, "100"));
Map<String, String> partition2 = new HashMap<>();
partition2.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition2.put(SinkUtils.KAFKA_PARTITION_KEY, "20");
@ -2464,7 +2463,7 @@ public class WorkerTest {
Map<String, String> partition1 = new HashMap<>();
partition1.put(SinkUtils.KAFKA_TOPIC_KEY, "test_topic");
partition1.put(SinkUtils.KAFKA_PARTITION_KEY, "10");
partitionOffsets.put(partition1, Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, "100"));
partitionOffsets.put(partition1, Map.of(SinkUtils.KAFKA_OFFSET_KEY, "100"));
FutureCallback<Message> cb = new FutureCallback<>();
worker.modifySinkConnectorOffsets(CONNECTOR_ID, sinkConnector, connectorProps, partitionOffsets,
@ -2504,8 +2503,8 @@ public class WorkerTest {
OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
Set<Map<String, Object>> connectorPartitions = new HashSet<>();
connectorPartitions.add(Collections.singletonMap("partitionKey", "partitionValue1"));
connectorPartitions.add(Collections.singletonMap("partitionKey", "partitionValue2"));
connectorPartitions.add(Map.of("partitionKey", "partitionValue1"));
connectorPartitions.add(Map.of("partitionKey", "partitionValue2"));
when(offsetStore.connectorPartitions(eq(CONNECTOR_ID))).thenReturn(connectorPartitions);
when(offsetWriter.doFlush(any())).thenAnswer(invocation -> {
invocation.getArgument(0, Callback.class).onCompletion(null, null);
@ -2546,7 +2545,7 @@ public class WorkerTest {
when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
TopicPartition tp = new TopicPartition("test-topic", 0);
mockAdminListConsumerGroupOffsets(admin, Collections.singletonMap(tp, new OffsetAndMetadata(10L)), null, time, 2000);
mockAdminListConsumerGroupOffsets(admin, Map.of(tp, new OffsetAndMetadata(10L)), null, time, 2000);
when(sinkConnector.alterOffsets(eq(connectorProps), eq(Collections.singletonMap(tp, null)))).thenAnswer(invocation -> {
time.sleep(3000);
return true;
@ -2588,7 +2587,7 @@ public class WorkerTest {
when(plugins.withClassLoader(any(ClassLoader.class), any(Runnable.class))).thenAnswer(AdditionalAnswers.returnsSecondArg());
TopicPartition tp = new TopicPartition("test-topic", 0);
mockAdminListConsumerGroupOffsets(admin, Collections.singletonMap(tp, new OffsetAndMetadata(10L)), null);
mockAdminListConsumerGroupOffsets(admin, Map.of(tp, new OffsetAndMetadata(10L)), null);
when(sinkConnector.alterOffsets(eq(connectorProps), eq(Collections.singletonMap(tp, null)))).thenReturn(true);
DeleteConsumerGroupsResult deleteConsumerGroupsResult = mock(DeleteConsumerGroupsResult.class);
@ -2629,9 +2628,9 @@ public class WorkerTest {
KafkaProducer<byte[], byte[]> producer = mock(KafkaProducer.class);
OffsetStorageWriter offsetWriter = mock(OffsetStorageWriter.class);
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = Collections.singletonMap(
Collections.singletonMap("partitionKey", "partitionValue"),
Collections.singletonMap("offsetKey", "offsetValue"));
Map<Map<String, ?>, Map<String, ?>> partitionOffsets = Map.of(
Map.of("partitionKey", "partitionValue"),
Map.of("offsetKey", "offsetValue"));
FutureCallback<Message> cb = new FutureCallback<>();
worker.modifySourceConnectorOffsets(CONNECTOR_ID, sourceConnector, connectorProps, partitionOffsets, offsetStore, producer,
@ -2715,7 +2714,7 @@ public class WorkerTest {
Map<String, String> taskConfig = new HashMap<>();
// No warnings or exceptions when a connector generates an empty list of task configs
when(sourceConnector.taskConfigs(1)).thenReturn(Collections.emptyList());
when(sourceConnector.taskConfigs(1)).thenReturn(List.of());
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) {
connectorProps.put(TASKS_MAX_CONFIG, "1");
List<Map<String, String>> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps));
@ -2724,9 +2723,9 @@ public class WorkerTest {
}
// No warnings or exceptions when a connector generates the maximum permitted number of task configs
when(sourceConnector.taskConfigs(1)).thenReturn(Collections.singletonList(taskConfig));
when(sourceConnector.taskConfigs(2)).thenReturn(Arrays.asList(taskConfig, taskConfig));
when(sourceConnector.taskConfigs(3)).thenReturn(Arrays.asList(taskConfig, taskConfig, taskConfig));
when(sourceConnector.taskConfigs(1)).thenReturn(List.of(taskConfig));
when(sourceConnector.taskConfigs(2)).thenReturn(List.of(taskConfig, taskConfig));
when(sourceConnector.taskConfigs(3)).thenReturn(List.of(taskConfig, taskConfig, taskConfig));
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) {
connectorProps.put(TASKS_MAX_CONFIG, "1");
List<Map<String, String>> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps));
@ -2740,12 +2739,12 @@ public class WorkerTest {
taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps));
assertEquals(3, taskConfigs.size());
assertEquals(Collections.emptyList(), logCaptureAppender.getMessages("WARN"));
assertEquals(Collections.emptyList(), logCaptureAppender.getMessages("ERROR"));
assertEquals(List.of(), logCaptureAppender.getMessages("WARN"));
assertEquals(List.of(), logCaptureAppender.getMessages("ERROR"));
}
// Warning/exception when a connector generates too many task configs
List<Map<String, String>> tooManyTaskConfigs = Arrays.asList(taskConfig, taskConfig, taskConfig, taskConfig);
List<Map<String, String>> tooManyTaskConfigs = List.of(taskConfig, taskConfig, taskConfig, taskConfig);
when(sourceConnector.taskConfigs(1)).thenReturn(tooManyTaskConfigs);
when(sourceConnector.taskConfigs(2)).thenReturn(tooManyTaskConfigs);
when(sourceConnector.taskConfigs(3)).thenReturn(tooManyTaskConfigs);
@ -2780,19 +2779,19 @@ public class WorkerTest {
);
// Regardless of enforcement, there should never be any error-level log messages
assertEquals(Collections.emptyList(), logCaptureAppender.getMessages("ERROR"));
assertEquals(List.of(), logCaptureAppender.getMessages("ERROR"));
}
}
// One last sanity check in case the connector is reconfigured and respects tasks.max
when(sourceConnector.taskConfigs(1)).thenReturn(Collections.singletonList(taskConfig));
when(sourceConnector.taskConfigs(1)).thenReturn(List.of(taskConfig));
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(Worker.class)) {
connectorProps.put(TASKS_MAX_CONFIG, "1");
List<Map<String, String>> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, new ConnectorConfig(plugins, connectorProps));
assertEquals(1, taskConfigs.size());
assertEquals(Collections.emptyList(), logCaptureAppender.getMessages("WARN"));
assertEquals(Collections.emptyList(), logCaptureAppender.getMessages("ERROR"));
assertEquals(List.of(), logCaptureAppender.getMessages("WARN"));
assertEquals(List.of(), logCaptureAppender.getMessages("ERROR"));
}
worker.stop();
@ -2816,7 +2815,7 @@ public class WorkerTest {
SinkTask task = mock(TestSinkTask.class);
mockKafkaClusterId();
Map<String, String> origProps = Collections.singletonMap(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
Map<String, String> origProps = Map.of(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName());
worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService,
noneConnectorClientConfigOverridePolicy, null);
@ -2824,7 +2823,7 @@ public class WorkerTest {
worker.start();
assertStatistics(worker, 0, 0);
assertEquals(Collections.emptySet(), worker.taskIds());
assertEquals(Set.of(), worker.taskIds());
Map<String, String> connectorConfigs = anyConnectorConfigMap();
connectorConfigs.put(TASKS_MAX_ENFORCE_CONFIG, Boolean.toString(enforced));
connectorConfigs.put(TOPICS_CONFIG, "t1");
@ -2839,15 +2838,15 @@ public class WorkerTest {
0,
null,
// ... but it has generated two task configs
Collections.singletonMap(connName, numTasks),
Collections.singletonMap(connName, connectorConfigs),
Collections.singletonMap(connName, TargetState.STARTED),
Collections.singletonMap(TASK_ID, origProps),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(connName, new AppliedConnectorConfig(connectorConfigs)),
Collections.emptySet(),
Collections.emptySet()
Map.of(connName, numTasks),
Map.of(connName, connectorConfigs),
Map.of(connName, TargetState.STARTED),
Map.of(TASK_ID, origProps),
Map.of(),
Map.of(),
Map.of(connName, new AppliedConnectorConfig(connectorConfigs)),
Set.of(),
Set.of()
);
String tasksMaxExceededMessage;
@ -2963,7 +2962,7 @@ public class WorkerTest {
private void mockInternalConverters() {
JsonConverter jsonConverter = new JsonConverter();
jsonConverter.configure(Collections.singletonMap(SCHEMAS_ENABLE_CONFIG, false), false);
jsonConverter.configure(Map.of(SCHEMAS_ENABLE_CONFIG, false), false);
when(plugins.newInternalConverter(eq(true), anyString(), anyMap()))
.thenReturn(jsonConverter);

View File

@ -31,10 +31,10 @@ import org.mockito.Mockito;
import org.mockito.stubbing.OngoingStubbing;
import java.util.AbstractMap.SimpleEntry;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@ -62,11 +62,11 @@ public class WorkerTestUtils {
connectorConfigs,
connectorTargetStates(1, connectorNum, TargetState.STARTED),
taskConfigs(0, connectorNum, connectorNum * taskNum),
Collections.emptyMap(),
Collections.emptyMap(),
Map.of(),
Map.of(),
appliedConnectorConfigs,
Collections.emptySet(),
Collections.emptySet());
Set.of(),
Set.of());
}
public static Map<String, Integer> connectorTaskCounts(int start,

View File

@ -21,8 +21,8 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
import org.junit.jupiter.api.Test;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V1;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2;
@ -99,48 +99,48 @@ public class ConnectProtocolCompatibilityTest {
public void testEagerToEagerAssignment() {
ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0));
List.of(connectorId1, connectorId3), List.of(taskId2x0));
ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0));
List.of(connectorId2), List.of(taskId1x0, taskId3x0));
ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf);
assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
}
@Test
public void testCoopToCoopAssignment() {
ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId1, connectorId3), List.of(taskId2x0),
List.of(), List.of(), 0);
ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ExtendedAssignment assignment2 = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId2), List.of(taskId1x0, taskId3x0),
List.of(), List.of(), 0);
ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment =
@ -148,15 +148,15 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
}
@Test
public void testEagerToCoopAssignment() {
ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0));
List.of(connectorId1, connectorId3), List.of(taskId2x0));
ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment);
ConnectProtocol.Assignment leaderAssignment =
@ -164,12 +164,12 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0));
List.of(connectorId2), List.of(taskId1x0, taskId3x0));
ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment =
@ -177,37 +177,37 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
}
@Test
public void testCoopToEagerAssignment() {
ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId1, connectorId3), List.of(taskId2x0),
List.of(), List.of(), 0);
ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ExtendedAssignment assignment2 = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId2), List.of(taskId1x0, taskId3x0),
List.of(), List.of(), 0);
ByteBuffer memberBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment2, false);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf);
assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
}
private ConnectProtocol.WorkerState emptyWorkerState() {
@ -221,10 +221,10 @@ public class ConnectProtocolCompatibilityTest {
LEADER,
LEADER_URL,
CONFIG_OFFSET,
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Set.of(),
Set.of(),
Set.of(),
Set.of(),
0
);
return new ExtendedWorkerState(LEADER_URL, CONFIG_OFFSET, assignment);

View File

@ -29,9 +29,6 @@ import org.mockito.quality.Strictness;
import java.security.InvalidParameterException;
import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -151,9 +148,9 @@ public class DistributedConfigTest {
private void testSupportedAlgorithms(String type, String... expectedAlgorithms) {
Set<String> supportedAlgorithms = DistributedConfig.supportedAlgorithms(type);
Set<String> unsupportedAlgorithms = new HashSet<>(Arrays.asList(expectedAlgorithms));
Set<String> unsupportedAlgorithms = new HashSet<>(List.of(expectedAlgorithms));
unsupportedAlgorithms.removeAll(supportedAlgorithms);
assertEquals(Collections.emptySet(), unsupportedAlgorithms, type + " algorithms were found that should be supported by this JVM but are not");
assertEquals(Set.of(), unsupportedAlgorithms, type + " algorithms were found that should be supported by this JVM but are not");
}
@Test
@ -214,13 +211,13 @@ public class DistributedConfigTest {
@Test
public void shouldValidateAllVerificationAlgorithms() {
List<String> algorithms =
new ArrayList<>(Arrays.asList("HmacSHA1", "HmacSHA256", "HmacMD5", "bad-algorithm"));
List<String> algorithms = List.of("HmacSHA1", "HmacSHA256", "HmacMD5", "bad-algorithm");
Map<String, String> configs = configs();
for (int i = 0; i < algorithms.size(); i++) {
configs.put(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG, String.join(",", algorithms));
assertThrows(ConfigException.class, () -> new DistributedConfig(configs));
algorithms.add(algorithms.remove(0));
// Rotate the algorithm list by creating a new list with rotated elements
algorithms = List.of(algorithms.get(1), algorithms.get(2), algorithms.get(3), algorithms.get(0));
}
}

View File

@ -37,9 +37,7 @@ import org.mockito.quality.Strictness;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -662,8 +660,8 @@ public class IncrementalCooperativeAssignorTest {
List<WorkerLoad> expectedAssignment = existingAssignment.stream()
.map(wl -> new WorkerLoad.Builder(wl.worker()).withCopies(wl.connectors(), wl.tasks()).build())
.collect(Collectors.toList());
expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10"));
expectedAssignment.get(0).connectors().addAll(List.of("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(List.of("connector7", "connector10"));
expectedAssignment.get(2).connectors().add("connector8");
List<String> newConnectors = newConnectors(6, 11);
@ -682,12 +680,12 @@ public class IncrementalCooperativeAssignorTest {
.map(wl -> new WorkerLoad.Builder(wl.worker()).withCopies(wl.connectors(), wl.tasks()).build())
.collect(Collectors.toList());
expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10"));
expectedAssignment.get(0).connectors().addAll(List.of("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(List.of("connector7", "connector10"));
expectedAssignment.get(2).connectors().add("connector8");
expectedAssignment.get(0).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 6), new ConnectorTaskId("task", 9)));
expectedAssignment.get(1).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 7), new ConnectorTaskId("task", 10)));
expectedAssignment.get(0).tasks().addAll(List.of(new ConnectorTaskId("task", 6), new ConnectorTaskId("task", 9)));
expectedAssignment.get(1).tasks().addAll(List.of(new ConnectorTaskId("task", 7), new ConnectorTaskId("task", 10)));
expectedAssignment.get(2).tasks().add(new ConnectorTaskId("task", 8));
List<String> newConnectors = newConnectors(6, 11);
@ -734,7 +732,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -751,7 +749,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -766,7 +764,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(flakyWorker),
assertEquals(Set.of(flakyWorker),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -787,7 +785,7 @@ public class IncrementalCooperativeAssignorTest {
.tasks()
.containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -814,7 +812,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -831,7 +829,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -845,7 +843,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -861,7 +859,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors");
assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -888,7 +886,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -908,7 +906,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(newWorker),
assertEquals(Set.of(newWorker),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -956,7 +954,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors");
assertTrue(listOfTasksInLast2Workers.containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -983,7 +981,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -1000,7 +998,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -1015,7 +1013,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(veryFlakyWorker),
assertEquals(Set.of(veryFlakyWorker),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -1034,7 +1032,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors");
assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -1062,7 +1060,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -1080,7 +1078,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, lostAssignmentsToReassign,
new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(),
assertEquals(Set.of(),
assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance);
@ -1221,10 +1219,10 @@ public class IncrementalCooperativeAssignorTest {
leader,
"followMe:618",
CONFIG_OFFSET,
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Set.of(),
Set.of(),
Set.of(),
Set.of(),
0
);
ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment);
@ -1262,10 +1260,10 @@ public class IncrementalCooperativeAssignorTest {
leader,
"followMe:618",
CONFIG_OFFSET,
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Collections.emptySet(),
Set.of(),
Set.of(),
Set.of(),
Set.of(),
0
);
ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment);
@ -1328,7 +1326,7 @@ public class IncrementalCooperativeAssignorTest {
private void addNewEmptyWorkers(String... workers) {
for (String worker : workers) {
addNewWorker(worker, Collections.emptyList(), Collections.emptyList());
addNewWorker(worker, List.of(), List.of());
}
}
@ -1392,13 +1390,13 @@ public class IncrementalCooperativeAssignorTest {
private ClusterConfigState configState() {
Map<String, Integer> taskCounts = new HashMap<>(connectors);
Map<String, Map<String, String>> connectorConfigs = transformValues(taskCounts, c -> Collections.emptyMap());
Map<String, Map<String, String>> connectorConfigs = transformValues(taskCounts, c -> Map.of());
Map<String, TargetState> targetStates = transformValues(taskCounts, c -> TargetState.STARTED);
Map<ConnectorTaskId, Map<String, String>> taskConfigs = taskCounts.entrySet().stream()
.flatMap(e -> IntStream.range(0, e.getValue()).mapToObj(i -> new ConnectorTaskId(e.getKey(), i)))
.collect(Collectors.toMap(
Function.identity(),
connectorTaskId -> Collections.emptyMap()
connectorTaskId -> Map.of()
));
Map<String, AppliedConnectorConfig> appliedConnectorConfigs = connectorConfigs.entrySet().stream()
.collect(Collectors.toMap(
@ -1412,11 +1410,11 @@ public class IncrementalCooperativeAssignorTest {
connectorConfigs,
targetStates,
taskConfigs,
Collections.emptyMap(),
Collections.emptyMap(),
Map.of(),
Map.of(),
appliedConnectorConfigs,
Collections.emptySet(),
Collections.emptySet());
Set.of(),
Set.of());
}
private void applyAssignments() {
@ -1440,16 +1438,16 @@ public class IncrementalCooperativeAssignorTest {
}
private void assertEmptyAssignment() {
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyAssignedConnectors().values()),
"No connectors should have been newly assigned during this round");
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyAssignedTasks().values()),
"No tasks should have been newly assigned during this round");
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyRevokedConnectors().values()),
"No connectors should have been revoked during this round");
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyRevokedTasks().values()),
"No tasks should have been revoked during this round");
}
@ -1500,14 +1498,14 @@ public class IncrementalCooperativeAssignorTest {
private void assertNoRevocations() {
returnedAssignments.newlyRevokedConnectors().forEach((worker, revocations) ->
assertEquals(
Collections.emptySet(),
Set.of(),
new HashSet<>(revocations),
"Expected no revocations to take place during this round, but connector revocations were issued for worker " + worker
)
);
returnedAssignments.newlyRevokedTasks().forEach((worker, revocations) ->
assertEquals(
Collections.emptySet(),
Set.of(),
new HashSet<>(revocations),
"Expected no revocations to take place during this round, but task revocations were issued for worker " + worker
)
@ -1542,11 +1540,11 @@ public class IncrementalCooperativeAssignorTest {
);
existingConnectors.retainAll(newConnectors);
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
existingConnectors,
"Found connectors in new assignment that already exist in current assignment");
existingTasks.retainAll(newTasks);
assertEquals(Collections.emptyList(),
assertEquals(List.of(),
existingConnectors,
"Found tasks in new assignment that already exist in current assignment");
}

View File

@ -42,9 +42,7 @@ import org.mockito.quality.Strictness;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -126,7 +124,7 @@ public class WorkerCoordinatorIncrementalTest {
this.time = new MockTime();
this.metadata = new Metadata(0, 0, Long.MAX_VALUE, loggerFactory, new ClusterResourceListeners());
this.client = new MockClient(time, metadata);
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Map.of("topic", 1)));
this.node = metadata.fetch().nodes().get(0);
this.consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time,
retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs);
@ -210,8 +208,8 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(),
Collections.singletonList(connectorId1), Arrays.asList(taskId1x0, taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId1), List.of(taskId1x0, taskId2x0),
List.of(), List.of(), 0);
ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
// Using onJoinComplete to register the protocol selection decided by the broker
// coordinator as well as an existing previous assignment that the call to metadata will
@ -229,8 +227,8 @@ public class WorkerCoordinatorIncrementalTest {
.deserializeMetadata(ByteBuffer.wrap(selectedMetadata.metadata()));
assertEquals(offset, state.offset());
assertNotEquals(ExtendedAssignment.empty(), state.assignment());
assertEquals(Collections.singletonList(connectorId1), state.assignment().connectors());
assertEquals(Arrays.asList(taskId1x0, taskId2x0), state.assignment().tasks());
assertEquals(List.of(connectorId1), state.assignment().connectors());
assertEquals(List.of(taskId1x0, taskId2x0), state.assignment().tasks());
verify(configStorage, times(1)).snapshot();
}
@ -243,8 +241,8 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(),
Collections.singletonList(connectorId1), Arrays.asList(taskId1x0, taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0);
List.of(connectorId1), List.of(taskId1x0, taskId2x0),
List.of(), List.of(), 0);
ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
// Using onJoinComplete to register the protocol selection decided by the broker
// coordinator as well as an existing previous assignment that the call to metadata will
@ -283,14 +281,14 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 4,
Collections.emptyList(), 0,
List.of(connectorId1), 4,
List.of(), 0,
leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 4,
Collections.emptyList(), 0,
List.of(connectorId2), 4,
List.of(), 0,
memberAssignment);
coordinator.metadata();
@ -306,20 +304,20 @@ public class WorkerCoordinatorIncrementalTest {
//Equally distributing tasks across member
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 1,
List.of(), 0,
List.of(), 1,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 1,
List.of(), 0,
List.of(), 1,
memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
anotherMemberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot();
@ -344,20 +342,20 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 3,
Collections.emptyList(), 0,
List.of(connectorId1), 3,
List.of(), 0,
leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 3,
Collections.emptyList(), 0,
List.of(connectorId2), 3,
List.of(), 0,
memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 2,
Collections.emptyList(), 0,
List.of(), 2,
List.of(), 0,
anotherMemberAssignment);
// Second rebalance detects a worker is missing
@ -373,15 +371,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
memberAssignment);
@ -393,15 +391,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
memberAssignment);
@ -412,14 +410,14 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 1,
Collections.emptyList(), 0,
List.of(), 1,
List.of(), 0,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 1,
Collections.emptyList(), 0,
List.of(), 1,
List.of(), 0,
memberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot();
@ -444,20 +442,20 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 3,
Collections.emptyList(), 0,
List.of(connectorId1), 3,
List.of(), 0,
leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 3,
Collections.emptyList(), 0,
List.of(connectorId2), 3,
List.of(), 0,
memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 2,
Collections.emptyList(), 0,
List.of(), 2,
List.of(), 0,
anotherMemberAssignment);
// Second rebalance detects a worker is missing
@ -472,15 +470,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
memberAssignment);
@ -494,22 +492,22 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
memberAssignment);
anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
rebalanceDelay,
anotherMemberAssignment);
@ -520,20 +518,20 @@ public class WorkerCoordinatorIncrementalTest {
// A rebalance after the delay expires re-assigns the lost tasks to the returning member
leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 0,
Collections.emptyList(), 0,
List.of(), 0,
List.of(), 0,
memberAssignment);
anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset,
Collections.emptyList(), 2,
Collections.emptyList(), 0,
List.of(), 2,
List.of(), 0,
anotherMemberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot();
@ -543,8 +541,8 @@ public class WorkerCoordinatorIncrementalTest {
public ExtendedAssignment assignment = null;
public String revokedLeader;
public Collection<String> revokedConnectors = Collections.emptyList();
public Collection<ConnectorTaskId> revokedTasks = Collections.emptyList();
public Collection<String> revokedConnectors = List.of();
public Collection<ConnectorTaskId> revokedTasks = List.of();
public int revokedCount = 0;
public int assignedCount = 0;

View File

@ -54,14 +54,13 @@ import org.mockito.quality.Strictness;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@ -128,7 +127,7 @@ public class WorkerCoordinatorTest {
this.time = new MockTime();
this.metadata = new Metadata(0, 0, Long.MAX_VALUE, logContext, new ClusterResourceListeners());
this.client = new MockClient(time, metadata);
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Collections.singletonMap("topic", 1)));
this.client.updateMetadata(RequestTestUtils.metadataUpdateWith(1, Map.of("topic", 1)));
this.node = metadata.fetch().nodes().get(0);
this.consumerClient = new ConsumerNetworkClient(logContext, client, metadata, time, 100, 1000, heartbeatIntervalMs);
this.metrics = new Metrics(time);
@ -158,15 +157,15 @@ public class WorkerCoordinatorTest {
configState1 = new ClusterConfigState(
4L,
null,
Collections.singletonMap(connectorId1, 1),
Collections.singletonMap(connectorId1, new HashMap<>()),
Collections.singletonMap(connectorId1, TargetState.STARTED),
Collections.singletonMap(taskId1x0, new HashMap<>()),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet()
Map.of(connectorId1, 1),
Map.of(connectorId1, new HashMap<>()),
Map.of(connectorId1, TargetState.STARTED),
Map.of(taskId1x0, new HashMap<>()),
Map.of(),
Map.of(),
Map.of(),
Set.of(),
Set.of()
);
Map<String, Integer> configState2ConnectorTaskCounts = new HashMap<>();
@ -189,11 +188,11 @@ public class WorkerCoordinatorTest {
configState2ConnectorConfigs,
configState2TargetStates,
configState2TaskConfigs,
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptySet(),
Collections.emptySet()
Map.of(),
Map.of(),
Map.of(),
Set.of(),
Set.of()
);
Map<String, Integer> configStateSingleTaskConnectorsConnectorTaskCounts = new HashMap<>();
@ -224,11 +223,11 @@ public class WorkerCoordinatorTest {
configStateSingleTaskConnectorsConnectorConfigs,
configStateSingleTaskConnectorsTargetStates,
configStateSingleTaskConnectorsTaskConfigs,
Collections.emptyMap(),
Collections.emptyMap(),
Map.of(),
Map.of(),
appliedConnectorConfigs,
Collections.emptySet(),
Collections.emptySet()
Set.of(),
Set.of()
);
}
@ -281,8 +280,8 @@ public class WorkerCoordinatorTest {
return sync.data().memberId().equals(memberId) &&
sync.data().generationId() == 1 &&
sync.groupAssignments().containsKey(memberId);
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), Collections.singletonList(connectorId1),
Collections.emptyList(), Errors.NONE));
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), List.of(connectorId1),
List.of(), Errors.NONE));
coordinator.ensureActiveGroup();
assertFalse(coordinator.rejoinNeededOrPending());
@ -291,8 +290,8 @@ public class WorkerCoordinatorTest {
assertFalse(rebalanceListener.assignment.failed());
assertEquals(configState1.offset(), rebalanceListener.assignment.offset());
assertEquals("leader", rebalanceListener.assignment.leader());
assertEquals(Collections.singletonList(connectorId1), rebalanceListener.assignment.connectors());
assertEquals(Collections.emptyList(), rebalanceListener.assignment.tasks());
assertEquals(List.of(connectorId1), rebalanceListener.assignment.connectors());
assertEquals(List.of(), rebalanceListener.assignment.tasks());
verify(configStorage).snapshot();
}
@ -315,8 +314,8 @@ public class WorkerCoordinatorTest {
return sync.data().memberId().equals(memberId) &&
sync.data().generationId() == 1 &&
sync.data().assignments().isEmpty();
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), Collections.emptyList(),
Collections.singletonList(taskId1x0), Errors.NONE));
}, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), List.of(),
List.of(taskId1x0), Errors.NONE));
coordinator.ensureActiveGroup();
assertFalse(coordinator.rejoinNeededOrPending());
@ -324,8 +323,8 @@ public class WorkerCoordinatorTest {
assertEquals(1, rebalanceListener.assignedCount);
assertFalse(rebalanceListener.assignment.failed());
assertEquals(configState1.offset(), rebalanceListener.assignment.offset());
assertEquals(Collections.emptyList(), rebalanceListener.assignment.connectors());
assertEquals(Collections.singletonList(taskId1x0), rebalanceListener.assignment.tasks());
assertEquals(List.of(), rebalanceListener.assignment.connectors());
assertEquals(List.of(taskId1x0), rebalanceListener.assignment.tasks());
verify(configStorage).snapshot();
}
@ -352,14 +351,14 @@ public class WorkerCoordinatorTest {
sync.data().assignments().isEmpty();
};
client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.CONFIG_MISMATCH, "leader", configState2.offset(),
Collections.emptyList(), Collections.emptyList(), Errors.NONE));
List.of(), List.of(), Errors.NONE));
// When the first round fails, we'll take an updated config snapshot
when(configStorage.snapshot()).thenReturn(configState2);
client.prepareResponse(joinGroupFollowerResponse(1, memberId, "leader", Errors.NONE));
client.prepareResponse(matcher, syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState2.offset(),
Collections.emptyList(), Collections.singletonList(taskId1x0), Errors.NONE));
List.of(), List.of(taskId1x0), Errors.NONE));
coordinator.ensureActiveGroup();
verify(configStorage, times(2)).snapshot();
@ -376,32 +375,32 @@ public class WorkerCoordinatorTest {
// join the group once
client.prepareResponse(joinGroupFollowerResponse(1, "member", "leader", Errors.NONE));
client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), Collections.emptyList(),
Collections.singletonList(taskId1x0), Errors.NONE));
client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), List.of(),
List.of(taskId1x0), Errors.NONE));
coordinator.ensureActiveGroup();
assertEquals(0, rebalanceListener.revokedCount);
assertEquals(1, rebalanceListener.assignedCount);
assertFalse(rebalanceListener.assignment.failed());
assertEquals(configState1.offset(), rebalanceListener.assignment.offset());
assertEquals(Collections.emptyList(), rebalanceListener.assignment.connectors());
assertEquals(Collections.singletonList(taskId1x0), rebalanceListener.assignment.tasks());
assertEquals(List.of(), rebalanceListener.assignment.connectors());
assertEquals(List.of(taskId1x0), rebalanceListener.assignment.tasks());
// and join the group again
coordinator.requestRejoin("test");
client.prepareResponse(joinGroupFollowerResponse(1, "member", "leader", Errors.NONE));
client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), Collections.singletonList(connectorId1),
Collections.emptyList(), Errors.NONE));
client.prepareResponse(syncGroupResponse(ConnectProtocol.Assignment.NO_ERROR, "leader", configState1.offset(), List.of(connectorId1),
List.of(), Errors.NONE));
coordinator.ensureActiveGroup();
assertEquals(1, rebalanceListener.revokedCount);
assertEquals(Collections.emptyList(), rebalanceListener.revokedConnectors);
assertEquals(Collections.singletonList(taskId1x0), rebalanceListener.revokedTasks);
assertEquals(List.of(), rebalanceListener.revokedConnectors);
assertEquals(List.of(taskId1x0), rebalanceListener.revokedTasks);
assertEquals(2, rebalanceListener.assignedCount);
assertFalse(rebalanceListener.assignment.failed());
assertEquals(configState1.offset(), rebalanceListener.assignment.offset());
assertEquals(Collections.singletonList(connectorId1), rebalanceListener.assignment.connectors());
assertEquals(Collections.emptyList(), rebalanceListener.assignment.tasks());
assertEquals(List.of(connectorId1), rebalanceListener.assignment.connectors());
assertEquals(List.of(), rebalanceListener.assignment.tasks());
verify(configStorage, times(2)).snapshot();
}
@ -435,15 +434,15 @@ public class WorkerCoordinatorTest {
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(configState1.offset(), leaderAssignment.offset());
assertEquals(Collections.singletonList(connectorId1), leaderAssignment.connectors());
assertEquals(Collections.emptyList(), leaderAssignment.tasks());
assertEquals(List.of(connectorId1), leaderAssignment.connectors());
assertEquals(List.of(), leaderAssignment.tasks());
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(result.get("member"));
assertFalse(memberAssignment.failed());
assertEquals("leader", memberAssignment.leader());
assertEquals(configState1.offset(), memberAssignment.offset());
assertEquals(Collections.emptyList(), memberAssignment.connectors());
assertEquals(Collections.singletonList(taskId1x0), memberAssignment.tasks());
assertEquals(List.of(), memberAssignment.connectors());
assertEquals(List.of(taskId1x0), memberAssignment.tasks());
verify(configStorage).snapshot();
}
@ -478,15 +477,15 @@ public class WorkerCoordinatorTest {
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(configState2.offset(), leaderAssignment.offset());
assertEquals(Collections.singletonList(connectorId1), leaderAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1), leaderAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(result.get("member"));
assertFalse(memberAssignment.failed());
assertEquals("leader", memberAssignment.leader());
assertEquals(configState2.offset(), memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Collections.singletonList(taskId1x1), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x1), memberAssignment.tasks());
verify(configStorage).snapshot();
}
@ -522,15 +521,15 @@ public class WorkerCoordinatorTest {
assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader());
assertEquals(configStateSingleTaskConnectors.offset(), leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks());
assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(result.get("member"));
assertFalse(memberAssignment.failed());
assertEquals("leader", memberAssignment.leader());
assertEquals(configStateSingleTaskConnectors.offset(), memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks());
assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
verify(configStorage).snapshot();
}
@ -547,7 +546,7 @@ public class WorkerCoordinatorTest {
coordinator.metadata();
assertThrows(IllegalStateException.class,
() -> coordinator.onLeaderElected("leader", EAGER.protocol(), Collections.emptyList(), true));
() -> coordinator.onLeaderElected("leader", EAGER.protocol(), List.of(), true));
verify(configStorage).snapshot();
}
@ -583,7 +582,7 @@ public class WorkerCoordinatorTest {
.setProtocolName(EAGER.protocol())
.setLeader(leaderId)
.setMemberId(memberId)
.setMembers(Collections.emptyList()),
.setMembers(List.of()),
ApiKeys.JOIN_GROUP.latestVersion()
);
}

View File

@ -46,8 +46,6 @@ import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonMap;
import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_CONNECTOR_NAME;
import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION;
import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE;
@ -105,13 +103,13 @@ public class ErrorReporterTest {
@Test
public void initializeDLQWithNullMetrics() {
assertThrows(NullPointerException.class, () -> new DeadLetterQueueReporter(producer, config(emptyMap()), TASK_ID, null));
assertThrows(NullPointerException.class, () -> new DeadLetterQueueReporter(producer, config(Map.of()), TASK_ID, null));
}
@Test
public void testDLQConfigWithEmptyTopicName() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(
producer, config(emptyMap()), TASK_ID, errorHandlingMetrics);
producer, config(Map.of()), TASK_ID, errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -124,7 +122,7 @@ public class ErrorReporterTest {
@Test
public void testDLQConfigWithValidTopicName() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(
producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
producer, config(Map.of(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -138,7 +136,7 @@ public class ErrorReporterTest {
@Test
public void testReportDLQTwice() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(
producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
producer, config(Map.of(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -153,7 +151,7 @@ public class ErrorReporterTest {
@Test
public void testCloseDLQ() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(
producer, config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
producer, config(Map.of(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC)), TASK_ID, errorHandlingMetrics);
deadLetterQueueReporter.close();
verify(producer).close();
@ -161,7 +159,7 @@ public class ErrorReporterTest {
@Test
public void testLogOnDisabledLogReporter() {
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(emptyMap()), errorHandlingMetrics);
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(Map.of()), errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
context.error(new RuntimeException());
@ -173,7 +171,7 @@ public class ErrorReporterTest {
@Test
public void testLogOnEnabledLogReporter() {
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics);
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(Map.of(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
context.error(new RuntimeException());
@ -185,7 +183,7 @@ public class ErrorReporterTest {
@Test
public void testLogMessageWithNoRecords() {
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(singletonMap(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics);
LogReporter<ConsumerRecord<byte[], byte[]>> logReporter = new LogReporter.Sink(TASK_ID, config(Map.of(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true")), errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -231,10 +229,10 @@ public class ErrorReporterTest {
@Test
public void testSetDLQConfigs() {
SinkConnectorConfig configuration = config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC));
SinkConnectorConfig configuration = config(Map.of(SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC));
assertEquals(DLQ_TOPIC, configuration.dlqTopicName());
configuration = config(singletonMap(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "7"));
configuration = config(Map.of(SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "7"));
assertEquals(7, configuration.dlqTopicReplicationFactor());
}

View File

@ -44,8 +44,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import org.mockito.stubbing.OngoingStubbing;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -54,11 +52,8 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonMap;
import static org.apache.kafka.common.utils.Time.SYSTEM;
import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_MAX_DELAY_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_MAX_DELAY_DEFAULT;
@ -232,22 +227,22 @@ public class RetryWithToleranceOperatorTest {
@Test
public void testExecAndHandleRetriableErrorOnce() throws Exception {
execAndHandleRetriableError(6000, 1, Collections.singletonList(300L), new RetriableException("Test"), true);
execAndHandleRetriableError(6000, 1, List.of(300L), new RetriableException("Test"), true);
}
@Test
public void testExecAndHandleRetriableErrorThrice() throws Exception {
execAndHandleRetriableError(6000, 3, Arrays.asList(300L, 600L, 1200L), new RetriableException("Test"), true);
execAndHandleRetriableError(6000, 3, List.of(300L, 600L, 1200L), new RetriableException("Test"), true);
}
@Test
public void testExecAndHandleRetriableErrorWithInfiniteRetries() throws Exception {
execAndHandleRetriableError(-1, 8, Arrays.asList(300L, 600L, 1200L, 2400L, 4800L, 9600L, 19200L, 38400L), new RetriableException("Test"), true);
execAndHandleRetriableError(-1, 8, List.of(300L, 600L, 1200L, 2400L, 4800L, 9600L, 19200L, 38400L), new RetriableException("Test"), true);
}
@Test
public void testExecAndHandleRetriableErrorWithMaxRetriesExceeded() throws Exception {
execAndHandleRetriableError(6000, 6, Arrays.asList(300L, 600L, 1200L, 2400L, 1500L), new RetriableException("Test"), false);
execAndHandleRetriableError(6000, 6, List.of(300L, 600L, 1200L, 2400L, 1500L), new RetriableException("Test"), false);
}
public void execAndHandleRetriableError(long errorRetryTimeout, int numRetriableExceptionsThrown, List<Long> expectedWaits, Exception e, boolean successExpected) throws Exception {
@ -396,7 +391,7 @@ public class RetryWithToleranceOperatorTest {
@Test
public void testDefaultConfigs() {
ConnectorConfig configuration = config(emptyMap());
ConnectorConfig configuration = config(Map.of());
assertEquals(ERRORS_RETRY_TIMEOUT_DEFAULT, configuration.errorRetryTimeout());
assertEquals(ERRORS_RETRY_MAX_DELAY_DEFAULT, configuration.errorMaxDelayInMillis());
assertEquals(ERRORS_TOLERANCE_DEFAULT, configuration.errorToleranceType());
@ -413,13 +408,13 @@ public class RetryWithToleranceOperatorTest {
@Test
public void testSetConfigs() {
ConnectorConfig configuration;
configuration = config(singletonMap(ERRORS_RETRY_TIMEOUT_CONFIG, "100"));
configuration = config(Map.of(ERRORS_RETRY_TIMEOUT_CONFIG, "100"));
assertEquals(100, configuration.errorRetryTimeout());
configuration = config(singletonMap(ERRORS_RETRY_MAX_DELAY_CONFIG, "100"));
configuration = config(Map.of(ERRORS_RETRY_MAX_DELAY_CONFIG, "100"));
assertEquals(100, configuration.errorMaxDelayInMillis());
configuration = config(singletonMap(ERRORS_TOLERANCE_CONFIG, "none"));
configuration = config(Map.of(ERRORS_TOLERANCE_CONFIG, "none"));
assertEquals(ToleranceType.NONE, configuration.errorToleranceType());
}
@ -439,7 +434,7 @@ public class RetryWithToleranceOperatorTest {
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator = new RetryWithToleranceOperator<>(-1, ERRORS_RETRY_MAX_DELAY_DEFAULT, ALL, time, errorHandlingMetrics, exitLatch);
ConsumerRecord<byte[], byte[]> consumerRecord = new ConsumerRecord<>("t", 0, 0, null, null);
List<CompletableFuture<RecordMetadata>> fs = IntStream.range(0, numberOfReports).mapToObj(i -> new CompletableFuture<RecordMetadata>()).toList();
List<ErrorReporter<ConsumerRecord<byte[], byte[]>>> reporters = IntStream.range(0, numberOfReports).mapToObj(i -> (ErrorReporter<ConsumerRecord<byte[], byte[]>>) c -> fs.get(i)).collect(Collectors.toList());
List<ErrorReporter<ConsumerRecord<byte[], byte[]>>> reporters = IntStream.range(0, numberOfReports).mapToObj(i -> (ErrorReporter<ConsumerRecord<byte[], byte[]>>) c -> fs.get(i)).toList();
retryWithToleranceOperator.reporters(reporters);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = new ProcessingContext<>(consumerRecord);
Future<Void> result = retryWithToleranceOperator.report(context);
@ -458,7 +453,7 @@ public class RetryWithToleranceOperatorTest {
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = allOperator();
retryWithToleranceOperator.reporters(Arrays.asList(reporterA, reporterB));
retryWithToleranceOperator.reporters(List.of(reporterA, reporterB));
// Even though the reporters throw exceptions, they should both still be closed.
@ -475,7 +470,7 @@ public class RetryWithToleranceOperatorTest {
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = allOperator();
retryWithToleranceOperator.reporters(Arrays.asList(reporterA, reporterB));
retryWithToleranceOperator.reporters(List.of(reporterA, reporterB));
// Even though the reporters throw exceptions, they should both still be closed.
doThrow(new RuntimeException()).when(reporterA).close();

View File

@ -35,7 +35,7 @@ import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import static org.junit.jupiter.api.Assertions.assertFalse;
@ -66,7 +66,7 @@ public class WorkerErrantRecordReporterTest {
for (int i = 0; i < 4; i++) {
TopicPartition topicPartition = new TopicPartition("topic", i);
topicPartitions.add(topicPartition);
reporter.futures.put(topicPartition, Collections.singletonList(CompletableFuture.completedFuture(null)));
reporter.futures.put(topicPartition, List.of(CompletableFuture.completedFuture(null)));
}
assertFalse(reporter.futures.isEmpty());
reporter.awaitFutures(topicPartitions);
@ -105,7 +105,7 @@ public class WorkerErrantRecordReporterTest {
Time.SYSTEM,
errorHandlingMetrics
);
retryWithToleranceOperator.reporters(Collections.singletonList(errorReporter));
retryWithToleranceOperator.reporters(List.of(errorReporter));
reporter = new WorkerErrantRecordReporter(
retryWithToleranceOperator,
converter,

View File

@ -29,9 +29,8 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@ -55,7 +54,7 @@ public class ConnectClusterStateImplTest {
@BeforeEach
public void setUp() {
expectedConnectors = Arrays.asList("sink1", "source1", "source2");
expectedConnectors = List.of("sink1", "source1", "source2");
connectClusterState = new ConnectClusterStateImpl(
herderRequestTimeoutMs,
new ConnectClusterDetailsImpl(KAFKA_CLUSTER_ID),
@ -78,7 +77,7 @@ public class ConnectClusterStateImplTest {
@Test
public void connectorConfig() {
final String connName = "sink6";
final Map<String, String> expectedConfig = Collections.singletonMap("key", "value");
final Map<String, String> expectedConfig = Map.of("key", "value");
@SuppressWarnings("unchecked")
ArgumentCaptor<Callback<Map<String, String>>> callback = ArgumentCaptor.forClass(Callback.class);

View File

@ -27,7 +27,6 @@ import org.mockito.quality.Strictness;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.Collections;
import java.util.SortedSet;
import java.util.TreeSet;
@ -74,14 +73,14 @@ public class DelegatingClassLoaderTest {
sinkConnectors.add(pluginDesc);
scanResult = new PluginScanResult(
sinkConnectors,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
}

View File

@ -30,7 +30,6 @@ import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -130,7 +129,7 @@ public class MultiVersionTest {
DEFAULT_COMBINED_ARTIFACT_VERSIONS.computeIfAbsent(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION, k -> "0.4.0"));
builder.include(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE,
DEFAULT_COMBINED_ARTIFACT_VERSIONS.computeIfAbsent(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE, k -> "0.5.0"));
DEFAULT_COMBINED_ARTIFACT = Collections.singletonMap(builder.build("all_versioned_artifact"), builder.buildInfos());
DEFAULT_COMBINED_ARTIFACT = Map.of(builder.build("all_versioned_artifact"), builder.buildInfos());
Map<Path, List<VersionedPluginBuilder.BuildInfo>> artifacts = new HashMap<>();
artifacts.putAll(DEFAULT_COMBINED_ARTIFACT);

View File

@ -24,7 +24,6 @@ import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -51,12 +50,12 @@ public class PluginRecommenderTest {
@Test
public void testConnectorVersionRecommenders() {
PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
for (String connectorClass : Arrays.asList(
for (String connectorClass : List.of(
VersionedPluginBuilder.VersionedTestPlugin.SINK_CONNECTOR.className(),
VersionedPluginBuilder.VersionedTestPlugin.SOURCE_CONNECTOR.className())
) {
Set<String> versions = recommender.connectorPluginVersionRecommender().validValues(
ConnectorConfig.CONNECTOR_CLASS_CONFIG, Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass)
ConnectorConfig.CONNECTOR_CLASS_CONFIG, Map.of(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass)
).stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(connectorClass);
Assertions.assertEquals(allVersions.size(), versions.size());
@ -73,7 +72,7 @@ public class PluginRecommenderTest {
config.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, converterClass);
config.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, converterClass);
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.CONVERTER.className());
for (ConfigDef.Recommender r : Arrays.asList(recommender.keyConverterPluginVersionRecommender(), recommender.valueConverterPluginVersionRecommender())) {
for (ConfigDef.Recommender r : List.of(recommender.keyConverterPluginVersionRecommender(), recommender.valueConverterPluginVersionRecommender())) {
Set<String> versions = r.validValues(null, config).stream().map(Object::toString).collect(Collectors.toSet());
Assertions.assertEquals(allVersions.size(), versions.size());
allVersions.forEach(v -> Assertions.assertTrue(versions.contains(v), "Missing version " + v + " for converter"));
@ -99,7 +98,7 @@ public class PluginRecommenderTest {
PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
Class transformationClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className());
Set<String> versions = recommender.transformationPluginRecommender("transforms.t1.type")
.validValues("transforms.t1.type", Collections.singletonMap("transforms.t1.type", transformationClass))
.validValues("transforms.t1.type", Map.of("transforms.t1.type", transformationClass))
.stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className());
Assertions.assertEquals(allVersions.size(), versions.size());
@ -112,7 +111,7 @@ public class PluginRecommenderTest {
PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
Class predicateClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className());
Set<String> versions = recommender.predicatePluginRecommender("predicates.p1.type")
.validValues("predicates.p1.type", Collections.singletonMap("predicates.p1.type", predicateClass))
.validValues("predicates.p1.type", Map.of("predicates.p1.type", predicateClass))
.stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className());
Assertions.assertEquals(allVersions.size(), versions.size());

View File

@ -26,7 +26,6 @@ import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Collections;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
@ -50,7 +49,7 @@ public class PluginScannerTest {
@ParameterizedTest
@MethodSource("parameters")
public void testScanningEmptyPluginPath(PluginScanner scanner) {
PluginScanResult result = scan(scanner, Collections.emptySet());
PluginScanResult result = scan(scanner, Set.of());
assertTrue(result.isEmpty());
}
@ -69,7 +68,7 @@ public class PluginScannerTest {
public void testScanningInvalidUberJar(PluginScanner scanner) throws Exception {
File newFile = new File(pluginDir, "invalid.jar");
newFile.createNewFile();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath()));
PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty());
}
@ -81,14 +80,14 @@ public class PluginScannerTest {
newFile = new File(newFile, "invalid.jar");
newFile.createNewFile();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath()));
PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty());
}
@ParameterizedTest
@MethodSource("parameters")
public void testScanningNoPlugins(PluginScanner scanner) {
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath()));
PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty());
}
@ -98,7 +97,7 @@ public class PluginScannerTest {
File newFile = new File(pluginDir, "my-plugin");
newFile.mkdir();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath()));
PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty());
}
@ -116,7 +115,7 @@ public class PluginScannerTest {
Files.copy(source, pluginPath.resolve(source.getFileName()));
}
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath()));
PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
Set<String> classes = new HashSet<>();
result.forEach(pluginDesc -> classes.add(pluginDesc.className()));
Set<String> expectedClasses = new HashSet<>(TestPlugins.pluginClasses());

View File

@ -113,7 +113,7 @@ public class PluginUtilsTest {
@Test
public void testConnectApiClasses() {
List<String> apiClasses = Arrays.asList(
List<String> apiClasses = List.of(
// Enumerate all packages and classes
"org.apache.kafka.connect.",
"org.apache.kafka.connect.components.",
@ -201,7 +201,7 @@ public class PluginUtilsTest {
@Test
public void testConnectRuntimeClasses() {
// Only list packages, because there are too many classes.
List<String> runtimeClasses = Arrays.asList(
List<String> runtimeClasses = List.of(
"org.apache.kafka.connect.cli.",
//"org.apache.kafka.connect.connector.policy.", isolated by default
//"org.apache.kafka.connect.converters.", isolated by default
@ -229,7 +229,7 @@ public class PluginUtilsTest {
@Test
public void testAllowedRuntimeClasses() {
List<String> jsonConverterClasses = Arrays.asList(
List<String> jsonConverterClasses = List.of(
"org.apache.kafka.connect.connector.policy.",
"org.apache.kafka.connect.connector.policy.AbstractConnectorClientConfigOverridePolicy",
"org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy",
@ -256,7 +256,7 @@ public class PluginUtilsTest {
@Test
public void testTransformsClasses() {
List<String> transformsClasses = Arrays.asList(
List<String> transformsClasses = List.of(
"org.apache.kafka.connect.transforms.",
"org.apache.kafka.connect.transforms.util.",
"org.apache.kafka.connect.transforms.util.NonEmptyListValidator",
@ -309,7 +309,7 @@ public class PluginUtilsTest {
@Test
public void testAllowedJsonConverterClasses() {
List<String> jsonConverterClasses = Arrays.asList(
List<String> jsonConverterClasses = List.of(
"org.apache.kafka.connect.json.",
"org.apache.kafka.connect.json.DecimalFormat",
"org.apache.kafka.connect.json.JsonConverter",
@ -326,7 +326,7 @@ public class PluginUtilsTest {
@Test
public void testAllowedFileConnectors() {
List<String> jsonConverterClasses = Arrays.asList(
List<String> jsonConverterClasses = List.of(
"org.apache.kafka.connect.file.",
"org.apache.kafka.connect.file.FileStreamSinkConnector",
"org.apache.kafka.connect.file.FileStreamSinkTask",
@ -341,7 +341,7 @@ public class PluginUtilsTest {
@Test
public void testAllowedBasicAuthExtensionClasses() {
List<String> basicAuthExtensionClasses = Arrays.asList(
List<String> basicAuthExtensionClasses = List.of(
"org.apache.kafka.connect.rest.basic.auth.extension.BasicAuthSecurityRestExtension"
//"org.apache.kafka.connect.rest.basic.auth.extension.JaasBasicAuthFilter", TODO fix?
//"org.apache.kafka.connect.rest.basic.auth.extension.PropertyFileLoginModule" TODO fix?
@ -377,13 +377,13 @@ public class PluginUtilsTest {
@Test
public void testEmptyPluginUrls() throws Exception {
assertEquals(Collections.emptyList(), PluginUtils.pluginUrls(pluginPath));
assertEquals(List.of(), PluginUtils.pluginUrls(pluginPath));
}
@Test
public void testEmptyStructurePluginUrls() throws Exception {
createBasicDirectoryLayout();
assertEquals(Collections.emptyList(), PluginUtils.pluginUrls(pluginPath));
assertEquals(List.of(), PluginUtils.pluginUrls(pluginPath));
}
@Test
@ -511,12 +511,12 @@ public class PluginUtilsTest {
sinkConnectors,
sourceConnectors,
converters,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
Map<String, String> actualAliases = PluginUtils.computeAliases(result);
Map<String, String> expectedAliases = new HashMap<>();
@ -538,14 +538,14 @@ public class PluginUtilsTest {
assertEquals(2, sinkConnectors.size());
PluginScanResult result = new PluginScanResult(
sinkConnectors,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
Map<String, String> actualAliases = PluginUtils.computeAliases(result);
Map<String, String> expectedAliases = new HashMap<>();
@ -561,15 +561,15 @@ public class PluginUtilsTest {
SortedSet<PluginDesc<HeaderConverter>> headerConverters = new TreeSet<>();
headerConverters.add(new PluginDesc<>(CollidingHeaderConverter.class, null, PluginType.HEADER_CONVERTER, CollidingHeaderConverter.class.getClassLoader()));
PluginScanResult result = new PluginScanResult(
Collections.emptySortedSet(),
Collections.emptySortedSet(),
new TreeSet<>(),
new TreeSet<>(),
converters,
headerConverters,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
Map<String, String> actualAliases = PluginUtils.computeAliases(result);
Map<String, String> expectedAliases = new HashMap<>();
@ -586,15 +586,15 @@ public class PluginUtilsTest {
SortedSet<PluginDesc<Transformation<?>>> transformations = new TreeSet<>();
transformations.add(new PluginDesc<>((Class<? extends Transformation<?>>) (Class<?>) Colliding.class, null, PluginType.TRANSFORMATION, Colliding.class.getClassLoader()));
PluginScanResult result = new PluginScanResult(
Collections.emptySortedSet(),
Collections.emptySortedSet(),
new TreeSet<>(),
new TreeSet<>(),
converters,
Collections.emptySortedSet(),
new TreeSet<>(),
transformations,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
Map<String, String> actualAliases = PluginUtils.computeAliases(result);
Map<String, String> expectedAliases = new HashMap<>();

View File

@ -55,14 +55,13 @@ import java.net.MalformedURLException;
import java.net.URL;
import java.net.URLClassLoader;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -107,16 +106,16 @@ public class PluginsTest {
missingPluginClass = sinkConnectors.first().className();
nonEmpty = new PluginScanResult(
sinkConnectors,
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet(),
Collections.emptySortedSet()
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>(),
new TreeSet<>()
);
empty = new PluginScanResult(Collections.emptyList());
empty = new PluginScanResult(List.of());
createConfig();
}
@ -140,7 +139,7 @@ public class PluginsTest {
@Test
public void shouldInstantiateAndConfigureInternalConverters() {
instantiateAndConfigureInternalConverter(true, Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"));
instantiateAndConfigureInternalConverter(true, Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"));
// Validate schemas.enable is set to false
assertEquals("false", internalConverter.configs.get(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG));
}
@ -207,7 +206,7 @@ public class PluginsTest {
public void shouldThrowIfPluginThrows() {
assertThrows(ConnectException.class, () -> plugins.newPlugin(
TestPlugin.ALWAYS_THROW_EXCEPTION.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
));
}
@ -216,7 +215,7 @@ public class PluginsTest {
public void shouldFindCoLocatedPluginIfBadPackaging() {
Converter converter = plugins.newPlugin(
TestPlugin.BAD_PACKAGING_CO_LOCATED.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
assertNotNull(converter);
@ -226,7 +225,7 @@ public class PluginsTest {
public void shouldThrowIfPluginMissingSuperclass() {
assertThrows(ConnectException.class, () -> plugins.newPlugin(
TestPlugin.BAD_PACKAGING_MISSING_SUPERCLASS.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
));
}
@ -242,7 +241,7 @@ public class PluginsTest {
public void shouldThrowIfStaticInitializerThrowsServiceLoader() {
assertThrows(ConnectException.class, () -> plugins.newPlugin(
TestPlugin.BAD_PACKAGING_STATIC_INITIALIZER_THROWS_REST_EXTENSION.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
ConnectRestExtension.class
));
}
@ -300,7 +299,7 @@ public class PluginsTest {
// Plugins are not isolated from other instances of their own class.
Converter firstPlugin = plugins.newPlugin(
TestPlugin.ALIASED_STATIC_FIELD.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
@ -308,7 +307,7 @@ public class PluginsTest {
Converter secondPlugin = plugins.newPlugin(
TestPlugin.ALIASED_STATIC_FIELD.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
@ -323,7 +322,7 @@ public class PluginsTest {
public void newPluginShouldServiceLoadWithPluginClassLoader() {
Converter plugin = plugins.newPlugin(
TestPlugin.SERVICE_LOADER.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
@ -339,7 +338,7 @@ public class PluginsTest {
public void newPluginShouldInstantiateWithPluginClassLoader() {
Converter plugin = plugins.newPlugin(
TestPlugin.ALIASED_STATIC_FIELD.className(),
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
@ -428,7 +427,7 @@ public class PluginsTest {
@Test
public void newPluginsShouldConfigureWithPluginClassLoader() {
List<Configurable> configurables = plugins.newPlugins(
Collections.singletonList(TestPlugin.SAMPLING_CONFIGURABLE.className()),
List.of(TestPlugin.SAMPLING_CONFIGURABLE.className()),
config,
Configurable.class
);
@ -596,7 +595,7 @@ public class PluginsTest {
String alias = "SamplingConverter";
assertTrue(TestPlugin.SAMPLING_CONVERTER.className().contains(alias));
ConfigDef def = new ConfigDef().define(configKey, ConfigDef.Type.CLASS, ConfigDef.Importance.HIGH, "docstring");
AbstractConfig config = new AbstractConfig(def, Collections.singletonMap(configKey, alias));
AbstractConfig config = new AbstractConfig(def, Map.of(configKey, alias));
assertNotNull(config.getClass(configKey));
assertNotNull(config.getConfiguredInstance(configKey, Converter.class));
@ -625,7 +624,7 @@ public class PluginsTest {
// Initialize Plugins object with parent class loader in the class loader tree. This is
// to simulate the situation where jars exist on both system classpath and plugin path.
Map<String, String> pluginProps = Collections.singletonMap(
Map<String, String> pluginProps = Map.of(
WorkerConfig.PLUGIN_PATH_CONFIG,
TestPlugins.pluginPathJoined(childResource)
);
@ -638,14 +637,14 @@ public class PluginsTest {
Converter converter = plugins.newPlugin(
className,
new AbstractConfig(new ConfigDef(), Collections.emptyMap()),
new AbstractConfig(new ConfigDef(), Map.of()),
Converter.class
);
// Verify the version was read from the correct resource
assertEquals(expectedVersions[0],
new String(converter.fromConnectData(null, null, null)));
// When requesting multiple resources, they should be listed in the correct order
assertEquals(Arrays.asList(expectedVersions),
assertEquals(List.of(expectedVersions),
converter.toConnectData(null, null).value());
}

View File

@ -17,7 +17,6 @@
package org.apache.kafka.connect.runtime.isolation;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -42,7 +41,7 @@ public interface SamplingTestPlugin {
* @return All known instances of this class, including this instance.
*/
default List<SamplingTestPlugin> allInstances() {
return Collections.singletonList(this);
return List.of(this);
}
/**
@ -50,7 +49,7 @@ public interface SamplingTestPlugin {
* This should only return direct children, and not reference this instance directly
*/
default Map<String, SamplingTestPlugin> otherSamples() {
return Collections.emptyMap();
return Map.of();
}
/**

View File

@ -43,7 +43,6 @@ import java.lang.management.MonitorInfo;
import java.lang.management.ThreadInfo;
import java.net.URL;
import java.util.Arrays;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.concurrent.BrokenBarrierException;
@ -71,7 +70,7 @@ public class SynchronizationTest {
@BeforeEach
public void setup(TestInfo testInfo) {
Map<String, String> pluginProps = Collections.singletonMap(
Map<String, String> pluginProps = Map.of(
WorkerConfig.PLUGIN_PATH_CONFIG,
TestPlugins.pluginPathJoined()
);
@ -241,7 +240,7 @@ public class SynchronizationTest {
// 4. Load the isolated plugin class and return
new AbstractConfig(
new ConfigDef().define("a.class", Type.CLASS, Importance.HIGH, ""),
Collections.singletonMap("a.class", t1Class));
Map.of("a.class", t1Class));
}
};
@ -259,7 +258,7 @@ public class SynchronizationTest {
// 3. Enter the DelegatingClassLoader
// 4. Load the non-isolated class and return
new AbstractConfig(new ConfigDef().define("a.class", Type.CLASS, Importance.HIGH, ""),
Collections.singletonMap("a.class", t2Class));
Map.of("a.class", t2Class));
}
};

View File

@ -298,7 +298,7 @@ public class TestPlugins {
if (pluginJars.containsKey(testPackage)) {
log.debug("Skipping recompilation of {}", testPackage.resourceDir());
}
pluginJars.put(testPackage, createPluginJar(testPackage.resourceDir(), testPackage.removeRuntimeClasses(), Collections.emptyMap()));
pluginJars.put(testPackage, createPluginJar(testPackage.resourceDir(), testPackage.removeRuntimeClasses(), Map.of()));
}
} catch (Throwable e) {
log.error("Could not set up plugin test jars", e);
@ -372,7 +372,7 @@ public class TestPlugins {
.filter(Objects::nonNull)
.map(TestPlugin::className)
.distinct()
.collect(Collectors.toList());
.toList();
}
public static Function<ClassLoader, LoaderSwap> noOpLoaderSwap() {
@ -458,11 +458,11 @@ public class TestPlugins {
.map(Path::toFile)
.filter(file -> file.getName().endsWith(".java"))
.map(file -> replacements.isEmpty() ? file : copyAndReplace(file, replacements))
.collect(Collectors.toList());
.toList();
}
StringWriter writer = new StringWriter();
List<String> options = Arrays.asList(
List<String> options = List.of(
"-d", binDir.toString() // Write class output to a different directory.
);
try (StandardJavaFileManager fileManager = compiler.getStandardFileManager(null, null, null)) {

View File

@ -21,8 +21,8 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
public class VersionedPluginBuilder {
@ -99,7 +99,7 @@ public class VersionedPluginBuilder {
Path subDir = Files.createDirectory(pluginDirPath.resolve("lib"));
subDir.toFile().deleteOnExit();
for (BuildInfo buildInfo : pluginBuilds) {
Path jarFile = TestPlugins.createPluginJar(buildInfo.plugin.resourceDir(), ignored -> false, Collections.singletonMap(VERSION_PLACEHOLDER, buildInfo.version));
Path jarFile = TestPlugins.createPluginJar(buildInfo.plugin.resourceDir(), ignored -> false, Map.of(VERSION_PLACEHOLDER, buildInfo.version));
Path targetJar = subDir.resolve(jarFile.getFileName()).toAbsolutePath();
buildInfo.setLocation(targetJar.toString());
targetJar.toFile().deleteOnExit();

View File

@ -60,9 +60,7 @@ import java.io.IOException;
import java.net.URI;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
@ -213,7 +211,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer();
@ -256,7 +254,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer();
@ -278,8 +276,8 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions();
doReturn(Collections.emptyList()).when(herder).setWorkerLoggerLevel(logger, loggingLevel);
doReturn(Collections.singletonMap(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels();
doReturn(List.of()).when(herder).setWorkerLoggerLevel(logger, loggingLevel);
doReturn(Map.of(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels();
server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer();
@ -296,7 +294,7 @@ public class ConnectRestServerTest {
Map<String, Object> expectedLogger = new HashMap<>();
expectedLogger.put("level", loggingLevel);
expectedLogger.put("last_modified", lastModified);
Map<String, Map<String, Object>> expectedLoggers = Collections.singletonMap(logger, expectedLogger);
Map<String, Map<String, Object>> expectedLoggers = Map.of(logger, expectedLogger);
Map<String, Map<String, Object>> actualLoggers = mapper.readValue(responseStr, new TypeReference<>() { });
assertEquals(expectedLoggers, actualLoggers);
}
@ -438,7 +436,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors();
doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer();
@ -481,8 +479,8 @@ public class ConnectRestServerTest {
}
private void expectEmptyRestExtensions() {
doReturn(Collections.emptyList()).when(plugins).newPlugins(
eq(Collections.emptyList()),
doReturn(List.of()).when(plugins).newPlugins(
eq(List.of()),
any(AbstractConfig.class),
eq(ConnectRestExtension.class)
);

View File

@ -21,8 +21,6 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -35,7 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class RestServerConfigTest {
private static final List<String> VALID_HEADER_CONFIGS = Arrays.asList(
private static final List<String> VALID_HEADER_CONFIGS = List.of(
"add \t Cache-Control: no-cache, no-store, must-revalidate",
"add \r X-XSS-Protection: 1; mode=block",
"\n add Strict-Transport-Security: max-age=31536000; includeSubDomains",
@ -48,7 +46,7 @@ public class RestServerConfigTest {
"adDdate \n Last-Modified: \t 0"
);
private static final List<String> INVALID_HEADER_CONFIGS = Arrays.asList(
private static final List<String> INVALID_HEADER_CONFIGS = List.of(
"set \t",
"badaction \t X-Frame-Options:DENY",
"set add X-XSS-Protection:1",
@ -70,11 +68,11 @@ public class RestServerConfigTest {
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999");
config = RestServerConfig.forPublic(null, props);
assertEquals(Collections.singletonList("http://a.b:9999"), config.listeners());
assertEquals(List.of("http://a.b:9999"), config.listeners());
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
config = RestServerConfig.forPublic(null, props);
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.listeners());
assertEquals(List.of("http://a.b:9999", "https://a.b:7812"), config.listeners());
}
@Test
@ -113,7 +111,7 @@ public class RestServerConfigTest {
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
config = RestServerConfig.forPublic(null, props);
assertEquals(Arrays.asList("http://a.b:9999", "https://a.b:7812"), config.adminListeners());
assertEquals(List.of("http://a.b:9999", "https://a.b:7812"), config.adminListeners());
RestServerConfig.forPublic(null, props);
}

View File

@ -18,8 +18,8 @@ package org.apache.kafka.connect.runtime.rest.entities;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -43,7 +43,7 @@ public class ConnectorOffsetsTest {
offset2.put("offset", new byte[]{0x00, 0x1A});
ConnectorOffset connectorOffset2 = new ConnectorOffset(partition2, offset2);
ConnectorOffsets connectorOffsets = new ConnectorOffsets(Arrays.asList(connectorOffset1, connectorOffset2));
ConnectorOffsets connectorOffsets = new ConnectorOffsets(List.of(connectorOffset1, connectorOffset2));
Map<Map<String, ?>, Map<String, ?>> connectorOffsetsMap = connectorOffsets.toMap();
assertEquals(2, connectorOffsetsMap.size());
assertEquals(offset1, connectorOffsetsMap.get(partition1));

View File

@ -20,7 +20,7 @@ import org.apache.kafka.connect.runtime.TargetState;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -33,7 +33,7 @@ public class CreateConnectorRequestTest {
assertEquals(TargetState.PAUSED, CreateConnectorRequest.InitialState.PAUSED.toTargetState());
assertEquals(TargetState.STOPPED, CreateConnectorRequest.InitialState.STOPPED.toTargetState());
CreateConnectorRequest createConnectorRequest = new CreateConnectorRequest("test-name", Collections.emptyMap(), null);
CreateConnectorRequest createConnectorRequest = new CreateConnectorRequest("test-name", Map.of(), null);
assertNull(createConnectorRequest.initialTargetState());
}

View File

@ -71,7 +71,6 @@ import org.mockito.quality.Strictness;
import java.net.URL;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
@ -86,7 +85,6 @@ import java.util.stream.Stream;
import jakarta.ws.rs.BadRequestException;
import static java.util.Arrays.asList;
import static org.apache.kafka.connect.runtime.rest.RestServer.DEFAULT_HEALTH_CHECK_TIMEOUT_MS;
import static org.apache.kafka.connect.runtime.rest.RestServer.DEFAULT_REST_REQUEST_TIMEOUT_MS;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -167,37 +165,37 @@ public class ConnectorPluginsResourceTest {
ConfigDef connectorConfigDef = ConnectorConfig.configDef();
List<ConfigValue> connectorConfigValues = connectorConfigDef.validate(PROPS);
List<ConfigValue> partialConnectorConfigValues = connectorConfigDef.validate(PARTIAL_PROPS);
ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), connectorConfigValues, Collections.emptyList());
ConfigInfos partialResult = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), partialConnectorConfigValues, Collections.emptyList());
ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), connectorConfigValues, List.of());
ConfigInfos partialResult = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), partialConnectorConfigValues, List.of());
List<ConfigInfo> configs = new LinkedList<>(result.values());
List<ConfigInfo> partialConfigs = new LinkedList<>(partialResult.values());
ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, null, "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.emptyList());
ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true);
ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, null, "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", List.of());
ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", List.of(), List.of(), true);
ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
configs.add(configInfo);
partialConfigs.add(configInfo);
configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, null, "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.emptyList());
configValueInfo = new ConfigValueInfo("test.int.config", "1", asList("1", "2", "3"), Collections.emptyList(), true);
configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, null, "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", List.of());
configValueInfo = new ConfigValueInfo("test.int.config", "1", List.of("1", "2", "3"), List.of(), true);
configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
configs.add(configInfo);
partialConfigs.add(configInfo);
configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.emptyList());
configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true);
configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", List.of());
configValueInfo = new ConfigValueInfo("test.string.config.default", "", List.of(), List.of(), true);
configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
configs.add(configInfo);
partialConfigs.add(configInfo);
configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, null, "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.emptyList());
configValueInfo = new ConfigValueInfo("test.list.config", "a,b", asList("a", "b", "c"), Collections.emptyList(), true);
configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, null, "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", List.of());
configValueInfo = new ConfigValueInfo("test.list.config", "a,b", List.of("a", "b", "c"), List.of(), true);
configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
configs.add(configInfo);
partialConfigs.add(configInfo);
CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), ERROR_COUNT, Collections.singletonList("Test"), configs);
PARTIAL_CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), PARTIAL_CONFIG_ERROR_COUNT, Collections.singletonList("Test"), partialConfigs);
CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), ERROR_COUNT, List.of("Test"), configs);
PARTIAL_CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), PARTIAL_CONFIG_ERROR_COUNT, List.of("Test"), partialConfigs);
}
private final Herder herder = mock(DistributedHerder.class);
@ -242,7 +240,7 @@ public class ConnectorPluginsResourceTest {
ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys,
configValues,
Collections.singletonList("Test")
List.of("Test")
);
configInfosCallback.getValue().onCompletion(null, configInfos);
return null;
@ -286,7 +284,7 @@ public class ConnectorPluginsResourceTest {
ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys,
configValues,
Collections.singletonList("Test")
List.of("Test")
);
configInfosCallback.getValue().onCompletion(null, configInfos);
return null;
@ -326,7 +324,7 @@ public class ConnectorPluginsResourceTest {
ConnectorPluginsResourceTestConnector.class.getName(),
resultConfigKeys,
configValues,
Collections.singletonList("Test")
List.of("Test")
);
configInfosCallback.getValue().onCompletion(null, configInfos);
return null;
@ -493,7 +491,7 @@ public class ConnectorPluginsResourceTest {
@Override
public List<Object> validValues(String name, Map<String, Object> parsedConfig) {
return asList(1, 2, 3);
return List.of(1, 2, 3);
}
@Override
@ -505,7 +503,7 @@ public class ConnectorPluginsResourceTest {
private static class ListRecommender implements Recommender {
@Override
public List<Object> validValues(String name, Map<String, Object> parsedConfig) {
return asList("a", "b", "c");
return List.of("a", "b", "c");
}
@Override

View File

@ -55,9 +55,7 @@ import org.mockito.stubbing.Stubber;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -142,14 +140,14 @@ public class ConnectorsResourceTest {
CONNECTOR_CONFIG_WITH_EMPTY_NAME.put(ConnectorConfig.NAME_CONFIG, "");
CONNECTOR_CONFIG_WITH_EMPTY_NAME.put("sample_config", "test_config");
}
private static final List<ConnectorTaskId> CONNECTOR_TASK_NAMES = Arrays.asList(
private static final List<ConnectorTaskId> CONNECTOR_TASK_NAMES = List.of(
new ConnectorTaskId(CONNECTOR_NAME, 0),
new ConnectorTaskId(CONNECTOR_NAME, 1)
);
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
static {
TASK_CONFIGS.add(Collections.singletonMap("config", "value"));
TASK_CONFIGS.add(Collections.singletonMap("config", "other_value"));
TASK_CONFIGS.add(Map.of("config", "value"));
TASK_CONFIGS.add(Map.of("config", "other_value"));
}
private static final List<TaskInfo> TASK_INFOS = new ArrayList<>();
static {
@ -158,7 +156,7 @@ public class ConnectorsResourceTest {
}
private static final Set<String> CONNECTOR_ACTIVE_TOPICS = new HashSet<>(
Arrays.asList("foo_topic", "bar_topic"));
List.of("foo_topic", "bar_topic"));
private static final RestRequestTimeout REQUEST_TIMEOUT = RestRequestTimeout.constant(
DEFAULT_REST_REQUEST_TIMEOUT_MS,
@ -196,7 +194,7 @@ public class ConnectorsResourceTest {
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>();
queryParams.putSingle("forward", "true");
when(forward.getQueryParameters()).thenReturn(queryParams);
when(herder.connectors()).thenReturn(Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME));
when(herder.connectors()).thenReturn(List.of(CONNECTOR2_NAME, CONNECTOR_NAME));
Collection<String> connectors = (Collection<String>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets
@ -205,7 +203,7 @@ public class ConnectorsResourceTest {
@Test
public void testExpandConnectorsStatus() {
when(herder.connectors()).thenReturn(Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME));
when(herder.connectors()).thenReturn(List.of(CONNECTOR2_NAME, CONNECTOR_NAME));
ConnectorStateInfo connector = mock(ConnectorStateInfo.class);
ConnectorStateInfo connector2 = mock(ConnectorStateInfo.class);
when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2);
@ -225,7 +223,7 @@ public class ConnectorsResourceTest {
@Test
public void testExpandConnectorsInfo() {
when(herder.connectors()).thenReturn(Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME));
when(herder.connectors()).thenReturn(List.of(CONNECTOR2_NAME, CONNECTOR_NAME));
ConnectorInfo connector = mock(ConnectorInfo.class);
ConnectorInfo connector2 = mock(ConnectorInfo.class);
when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connector2);
@ -245,7 +243,7 @@ public class ConnectorsResourceTest {
@Test
public void testFullExpandConnectors() {
when(herder.connectors()).thenReturn(Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME));
when(herder.connectors()).thenReturn(List.of(CONNECTOR2_NAME, CONNECTOR_NAME));
ConnectorInfo connectorInfo = mock(ConnectorInfo.class);
ConnectorInfo connectorInfo2 = mock(ConnectorInfo.class);
when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connectorInfo2);
@ -257,7 +255,7 @@ public class ConnectorsResourceTest {
forward = mock(UriInfo.class);
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>();
queryParams.put("expand", Arrays.asList("info", "status"));
queryParams.put("expand", List.of("info", "status"));
when(forward.getQueryParameters()).thenReturn(queryParams);
Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
@ -271,7 +269,7 @@ public class ConnectorsResourceTest {
@Test
public void testExpandConnectorsWithConnectorNotFound() {
when(herder.connectors()).thenReturn(Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME));
when(herder.connectors()).thenReturn(List.of(CONNECTOR2_NAME, CONNECTOR_NAME));
ConnectorStateInfo connector2 = mock(ConnectorStateInfo.class);
when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2);
doThrow(mock(NotFoundException.class)).when(herder).connectorStatus(CONNECTOR_NAME);
@ -283,7 +281,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(Collections.singleton(CONNECTOR2_NAME), expanded.keySet());
assertEquals(Set.of(CONNECTOR2_NAME), expanded.keySet());
assertEquals(connector2, expanded.get(CONNECTOR2_NAME).get("status"));
}
@ -291,7 +289,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnector() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -304,7 +302,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithPausedInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.PAUSED);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.PAUSED);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -317,7 +315,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithStoppedInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.STOPPED);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.STOPPED);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -330,7 +328,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithRunningInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.RUNNING);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), CreateConnectorRequest.InitialState.RUNNING);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -343,7 +341,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorNotLeader() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackNotLeaderException(cb).when(herder)
@ -357,7 +355,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithHeaders() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
HttpHeaders httpHeaders = mock(HttpHeaders.class);
expectAndCallbackNotLeaderException(cb)
@ -371,7 +369,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorExists() {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackException(cb, new AlreadyExistsException("already exists"))
@ -496,7 +494,7 @@ public class ConnectorsResourceTest {
connectorTask0Configs.put("connector-task1-config0", "321");
connectorTask0Configs.put("connector-task1-config1", "654");
final ConnectorTaskId connector2Task0 = new ConnectorTaskId(CONNECTOR2_NAME, 0);
final Map<String, String> connector2Task0Configs = Collections.singletonMap("connector2-task0-config0", "789");
final Map<String, String> connector2Task0Configs = Map.of("connector2-task0-config0", "789");
final List<TaskInfo> expectedTasksConnector = new ArrayList<>();
expectedTasksConnector.add(new TaskInfo(connectorTask0, connectorTask0Configs));
@ -529,7 +527,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithSpecialCharsInName() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME_SPECIAL_CHARS,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME_SPECIAL_CHARS), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME_SPECIAL_CHARS), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME_SPECIAL_CHARS, CONNECTOR_CONFIG,
@ -544,7 +542,7 @@ public class ConnectorsResourceTest {
@Test
public void testCreateConnectorWithControlSequenceInName() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME_CONTROL_SEQUENCES1,
Collections.singletonMap(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME_CONTROL_SEQUENCES1), null);
Map.of(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME_CONTROL_SEQUENCES1), null);
final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME_CONTROL_SEQUENCES1, CONNECTOR_CONFIG,
@ -690,7 +688,7 @@ public class ConnectorsResourceTest {
ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState(
AbstractStatus.State.RESTARTING.name(), "foo", null, null
);
ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, Collections.emptyList(), ConnectorType.SOURCE);
ConnectorStateInfo connectorStateInfo = new ConnectorStateInfo(CONNECTOR_NAME, state, List.of(), ConnectorType.SOURCE);
RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, true, false);
final ArgumentCaptor<Callback<ConnectorStateInfo>> cb = ArgumentCaptor.forClass(Callback.class);
@ -859,9 +857,9 @@ public class ConnectorsResourceTest {
@Test
public void testGetOffsets() throws Throwable {
final ArgumentCaptor<Callback<ConnectorOffsets>> cb = ArgumentCaptor.forClass(Callback.class);
ConnectorOffsets offsets = new ConnectorOffsets(Arrays.asList(
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue"))
ConnectorOffsets offsets = new ConnectorOffsets(List.of(
new ConnectorOffset(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
new ConnectorOffset(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"))
));
expectAndCallbackResult(cb, offsets).when(herder).connectorOffsets(eq(CONNECTOR_NAME), cb.capture());
@ -871,7 +869,7 @@ public class ConnectorsResourceTest {
@Test
public void testAlterOffsetsEmptyOffsets() {
assertThrows(BadRequestException.class, () -> connectorsResource.alterConnectorOffsets(
false, NULL_HEADERS, CONNECTOR_NAME, new ConnectorOffsets(Collections.emptyList())));
false, NULL_HEADERS, CONNECTOR_NAME, new ConnectorOffsets(List.of())));
}
@Test
@ -879,7 +877,7 @@ public class ConnectorsResourceTest {
Map<String, ?> partition = new HashMap<>();
Map<String, ?> offset = new HashMap<>();
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset);
ConnectorOffsets body = new ConnectorOffsets(Collections.singletonList(connectorOffset));
ConnectorOffsets body = new ConnectorOffsets(List.of(connectorOffset));
final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackNotLeaderException(cb).when(herder).alterConnectorOffsets(eq(CONNECTOR_NAME), eq(body.toMap()), cb.capture());
@ -894,7 +892,7 @@ public class ConnectorsResourceTest {
Map<String, ?> partition = new HashMap<>();
Map<String, ?> offset = new HashMap<>();
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset);
ConnectorOffsets body = new ConnectorOffsets(Collections.singletonList(connectorOffset));
ConnectorOffsets body = new ConnectorOffsets(List.of(connectorOffset));
final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackException(cb, new NotFoundException("Connector not found"))
.when(herder).alterConnectorOffsets(eq(CONNECTOR_NAME), eq(body.toMap()), cb.capture());
@ -904,10 +902,10 @@ public class ConnectorsResourceTest {
@Test
public void testAlterOffsets() throws Throwable {
Map<String, ?> partition = Collections.singletonMap("partitionKey", "partitionValue");
Map<String, ?> offset = Collections.singletonMap("offsetKey", "offsetValue");
Map<String, ?> partition = Map.of("partitionKey", "partitionValue");
Map<String, ?> offset = Map.of("offsetKey", "offsetValue");
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset);
ConnectorOffsets body = new ConnectorOffsets(Collections.singletonList(connectorOffset));
ConnectorOffsets body = new ConnectorOffsets(List.of(connectorOffset));
final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
Message msg = new Message("The offsets for this connector have been altered successfully");

View File

@ -39,7 +39,6 @@ import org.mockito.stubbing.Stubber;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Base64;
import java.util.Collections;
import java.util.List;
import java.util.Map;
@ -66,8 +65,8 @@ public class InternalConnectResourceTest {
private static final HttpHeaders NULL_HEADERS = null;
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
static {
TASK_CONFIGS.add(Collections.singletonMap("config", "value"));
TASK_CONFIGS.add(Collections.singletonMap("config", "other_value"));
TASK_CONFIGS.add(Map.of("config", "value"));
TASK_CONFIGS.add(Map.of("config", "other_value"));
}
private static final String FENCE_PATH = "/connectors/" + CONNECTOR_NAME + "/fence";
private static final String TASK_CONFIGS_PATH = "/connectors/" + CONNECTOR_NAME + "/tasks";

View File

@ -32,8 +32,8 @@ import org.mockito.quality.Strictness;
import org.slf4j.event.Level;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import jakarta.ws.rs.core.Response;
@ -90,7 +90,7 @@ public class LoggingResourceTest {
BadRequestException.class,
() -> loggingResource.setLevel(
"@root",
Collections.emptyMap(),
Map.of(),
scope
)
);
@ -104,7 +104,7 @@ public class LoggingResourceTest {
NotFoundException.class,
() -> loggingResource.setLevel(
"@root",
Collections.singletonMap("level", "HIGH"),
Map.of("level", "HIGH"),
scope
)
);
@ -130,7 +130,7 @@ public class LoggingResourceTest {
private void testSetLevelWorkerScope(String scope, boolean expectWarning) {
final String logger = "org.apache.kafka.connect";
final String level = "TRACE";
final List<String> expectedLoggers = Arrays.asList(
final List<String> expectedLoggers = List.of(
"org.apache.kafka.connect",
"org.apache.kafka.connect.runtime.distributed.DistributedHerder"
);
@ -138,7 +138,7 @@ public class LoggingResourceTest {
List<String> actualLoggers;
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(LoggingResource.class)) {
Response response = loggingResource.setLevel(logger, Collections.singletonMap("level", level), scope);
Response response = loggingResource.setLevel(logger, Map.of("level", level), scope);
assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
actualLoggers = (List<String>) response.getEntity();
long warningMessages = logCaptureAppender.getEvents().stream()
@ -159,7 +159,7 @@ public class LoggingResourceTest {
final String logger = "org.apache.kafka.connect";
final String level = "TRACE";
Response response = loggingResource.setLevel(logger, Collections.singletonMap("level", level), "cluster");
Response response = loggingResource.setLevel(logger, Map.of("level", level), "cluster");
assertEquals(Response.Status.NO_CONTENT.getStatusCode(), response.getStatus());
assertNull(response.getEntity());

View File

@ -25,8 +25,8 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
@ -156,7 +156,7 @@ public class SSLUtilsTest {
assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, ssl.getProtocol());
assertArrayEquals(Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols());
assertArrayEquals(List.of(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols());
assertEquals(SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm());
assertFalse(ssl.getNeedClientAuth());
@ -181,7 +181,7 @@ public class SSLUtilsTest {
assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, ssl.getProtocol());
assertArrayEquals(Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols());
assertArrayEquals(List.of(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split("\\s*,\\s*")).toArray(), ssl.getIncludeProtocols());
assertEquals(SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm());
}

View File

@ -75,20 +75,16 @@ import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static java.util.Collections.emptyList;
import static java.util.Collections.singleton;
import static java.util.Collections.singletonList;
import static java.util.Collections.singletonMap;
import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX;
import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG;
import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG;
@ -202,7 +198,7 @@ public class StandaloneHerderTest {
when(connectorMock.config()).thenReturn(new ConfigDef());
ConfigValue validatedValue = new ConfigValue("foo.bar");
when(connectorMock.validate(config)).thenReturn(new Config(singletonList(validatedValue)));
when(connectorMock.validate(config)).thenReturn(new Config(List.of(validatedValue)));
herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
@ -265,7 +261,7 @@ public class StandaloneHerderTest {
herder.putConnectorConfig(CONNECTOR_NAME, config, TargetState.STOPPED, false, createCallback);
Herder.Created<ConnectorInfo> connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
assertEquals(
new ConnectorInfo(CONNECTOR_NAME, connectorConfig(SourceSink.SINK), Collections.emptyList(), ConnectorType.SINK),
new ConnectorInfo(CONNECTOR_NAME, connectorConfig(SourceSink.SINK), List.of(), ConnectorType.SINK),
connectorInfo.result()
);
verify(loaderSwap).close();
@ -279,7 +275,7 @@ public class StandaloneHerderTest {
Map<String, String> config = connectorConfig(SourceSink.SOURCE);
expectConfigValidation(SourceSink.SOURCE, config);
when(statusBackingStore.getAll(CONNECTOR_NAME)).thenReturn(Collections.emptyList());
when(statusBackingStore.getAll(CONNECTOR_NAME)).thenReturn(List.of());
when(worker.connectorVersion(CONNECTOR_NAME)).thenReturn(null);
herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
@ -315,11 +311,11 @@ public class StandaloneHerderTest {
mockStartConnector(config, TargetState.STARTED, TargetState.STARTED, null);
when(worker.connectorNames()).thenReturn(Collections.singleton(CONNECTOR_NAME));
when(worker.connectorNames()).thenReturn(Set.of(CONNECTOR_NAME));
when(worker.getPlugins()).thenReturn(plugins);
// same task configs as earlier, so don't expect a new set of tasks to be brought up
when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, config, true)))
.thenReturn(Collections.singletonList(taskConfig(SourceSink.SOURCE)));
.thenReturn(List.of(taskConfig(SourceSink.SOURCE)));
herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
Herder.Created<ConnectorInfo> connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
@ -346,13 +342,13 @@ public class StandaloneHerderTest {
mockStartConnector(config, TargetState.STARTED, TargetState.STARTED, null);
when(worker.connectorNames()).thenReturn(Collections.singleton(CONNECTOR_NAME));
when(worker.connectorNames()).thenReturn(Set.of(CONNECTOR_NAME));
when(worker.getPlugins()).thenReturn(plugins);
// changed task configs, expect a new set of tasks to be brought up (and the old ones to be stopped)
Map<String, String> taskConfigs = taskConfig(SourceSink.SOURCE);
taskConfigs.put("k", "v");
when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, config, true)))
.thenReturn(Collections.singletonList(taskConfigs));
.thenReturn(List.of(taskConfigs));
when(worker.startSourceTask(eq(new ConnectorTaskId(CONNECTOR_NAME, 0)), any(), eq(connectorConfig(SourceSink.SOURCE)), eq(taskConfigs), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
@ -401,13 +397,13 @@ public class StandaloneHerderTest {
ClusterConfigState configState = new ClusterConfigState(
-1,
null,
Collections.singletonMap(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SOURCE)),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Map.of(CONNECTOR_NAME, 1),
Map.of(CONNECTOR_NAME, connectorConfig),
Map.of(CONNECTOR_NAME, TargetState.STARTED),
Map.of(taskId, taskConfig(SourceSink.SOURCE)),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(),
new HashSet<>(),
transformer);
@ -436,13 +432,13 @@ public class StandaloneHerderTest {
ClusterConfigState configState = new ClusterConfigState(
-1,
null,
Collections.singletonMap(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Map.of(CONNECTOR_NAME, 1),
Map.of(CONNECTOR_NAME, connectorConfig),
Map.of(CONNECTOR_NAME, TargetState.STARTED),
Map.of(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(),
new HashSet<>(),
transformer);
@ -560,7 +556,7 @@ public class StandaloneHerderTest {
when(restartPlan.shouldRestartTasks()).thenReturn(true);
when(restartPlan.restartTaskCount()).thenReturn(1);
when(restartPlan.totalTaskCount()).thenReturn(1);
when(restartPlan.taskIdsToRestart()).thenReturn(Collections.singletonList(taskId));
when(restartPlan.taskIdsToRestart()).thenReturn(List.of(taskId));
when(restartPlan.restartConnectorStateInfo()).thenReturn(connectorStateInfo);
doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest);
@ -570,18 +566,18 @@ public class StandaloneHerderTest {
Map<String, String> connectorConfig = connectorConfig(SourceSink.SINK);
expectConfigValidation(SourceSink.SINK, connectorConfig);
doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId));
doNothing().when(worker).stopAndAwaitTasks(List.of(taskId));
ClusterConfigState configState = new ClusterConfigState(
-1,
null,
Collections.singletonMap(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Map.of(CONNECTOR_NAME, 1),
Map.of(CONNECTOR_NAME, connectorConfig),
Map.of(CONNECTOR_NAME, TargetState.STARTED),
Map.of(taskId, taskConfig(SourceSink.SINK)),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(),
new HashSet<>(),
transformer);
@ -612,7 +608,7 @@ public class StandaloneHerderTest {
when(restartPlan.shouldRestartTasks()).thenReturn(true);
when(restartPlan.restartTaskCount()).thenReturn(1);
when(restartPlan.totalTaskCount()).thenReturn(1);
when(restartPlan.taskIdsToRestart()).thenReturn(Collections.singletonList(taskId));
when(restartPlan.taskIdsToRestart()).thenReturn(List.of(taskId));
when(restartPlan.restartConnectorStateInfo()).thenReturn(connectorStateInfo);
doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest);
@ -626,20 +622,20 @@ public class StandaloneHerderTest {
expectConfigValidation(SourceSink.SINK, connectorConfig);
doNothing().when(worker).stopAndAwaitConnector(CONNECTOR_NAME);
doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId));
doNothing().when(worker).stopAndAwaitTasks(List.of(taskId));
mockStartConnector(connectorConfig, null, TargetState.STARTED, null);
ClusterConfigState configState = new ClusterConfigState(
-1,
null,
Collections.singletonMap(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Map.of(CONNECTOR_NAME, 1),
Map.of(CONNECTOR_NAME, connectorConfig),
Map.of(CONNECTOR_NAME, TargetState.STARTED),
Map.of(taskId, taskConfig(SourceSink.SINK)),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(),
new HashSet<>(),
transformer);
@ -694,7 +690,7 @@ public class StandaloneHerderTest {
Callback<List<TaskInfo>> taskConfigsCb = mock(Callback.class);
// Check accessors with empty worker
doNothing().when(listConnectorsCb).onCompletion(null, Collections.EMPTY_SET);
doNothing().when(listConnectorsCb).onCompletion(null, Set.of());
doNothing().when(connectorInfoCb).onCompletion(any(NotFoundException.class), isNull());
doNothing().when(taskConfigsCb).onCompletion(any(NotFoundException.class), isNull());
doNothing().when(connectorConfigCb).onCompletion(any(NotFoundException.class), isNull());
@ -703,13 +699,13 @@ public class StandaloneHerderTest {
expectConfigValidation(SourceSink.SOURCE, connConfig);
// Validate accessors with 1 connector
doNothing().when(listConnectorsCb).onCompletion(null, singleton(CONNECTOR_NAME));
ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, singletonList(new ConnectorTaskId(CONNECTOR_NAME, 0)),
doNothing().when(listConnectorsCb).onCompletion(null, Set.of(CONNECTOR_NAME));
ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, List.of(new ConnectorTaskId(CONNECTOR_NAME, 0)),
ConnectorType.SOURCE);
doNothing().when(connectorInfoCb).onCompletion(null, connInfo);
TaskInfo taskInfo = new TaskInfo(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE));
doNothing().when(taskConfigsCb).onCompletion(null, singletonList(taskInfo));
doNothing().when(taskConfigsCb).onCompletion(null, List.of(taskInfo));
// All operations are synchronous for StandaloneHerder, so we don't need to actually wait after making each call
herder.connectors(listConnectorsCb);
@ -758,8 +754,8 @@ public class StandaloneHerderTest {
// Generate same task config, but from different connector config, resulting
// in task restarts
when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig, true)))
.thenReturn(singletonList(taskConfig(SourceSink.SOURCE)));
doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId));
.thenReturn(List.of(taskConfig(SourceSink.SOURCE)));
doNothing().when(worker).stopAndAwaitTasks(List.of(taskId));
doNothing().when(statusBackingStore).put(new TaskStatus(taskId, TaskStatus.State.DESTROYED, WORKER_ID, 0));
when(worker.startSourceTask(eq(taskId), any(), eq(newConnConfig), eq(taskConfig(SourceSink.SOURCE)), eq(herder), eq(TargetState.STARTED))).thenReturn(true);
@ -773,7 +769,7 @@ public class StandaloneHerderTest {
doNothing().when(connectorConfigCb).onCompletion(null, newConnConfig);
herder.putConnectorConfig(CONNECTOR_NAME, newConnConfig, true, reconfigureCallback);
Herder.Created<ConnectorInfo> newConnectorInfo = reconfigureCallback.get(1000L, TimeUnit.SECONDS);
ConnectorInfo newConnInfo = new ConnectorInfo(CONNECTOR_NAME, newConnConfig, singletonList(new ConnectorTaskId(CONNECTOR_NAME, 0)),
ConnectorInfo newConnInfo = new ConnectorInfo(CONNECTOR_NAME, newConnConfig, List.of(new ConnectorTaskId(CONNECTOR_NAME, 0)),
ConnectorType.SOURCE);
assertEquals(newConnInfo, newConnectorInfo.result());
@ -852,7 +848,7 @@ public class StandaloneHerderTest {
eq(herder), eq(TargetState.STARTED), onStart.capture());
ConnectorConfig connConfig = new SourceConnectorConfig(plugins, config, true);
when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig))
.thenReturn(emptyList());
.thenReturn(List.of());
}
@Test
@ -861,7 +857,7 @@ public class StandaloneHerderTest {
Callback<Void> cb = mock(Callback.class);
assertThrows(UnsupportedOperationException.class, () -> herder.putTaskConfigs(CONNECTOR_NAME,
singletonList(singletonMap("config", "value")), cb, null));
List.of(Map.of("config", "value")), cb, null));
}
@Test
@ -874,11 +870,11 @@ public class StandaloneHerderTest {
config.put(SinkConnectorConfig.TOPICS_CONFIG, TOPICS_LIST_STR);
Connector connectorMock = mock(SinkConnector.class);
String error = "This is an error in your config!";
List<String> errors = new ArrayList<>(singletonList(error));
List<String> errors = new ArrayList<>(List.of(error));
String key = "foo.invalid.key";
when(connectorMock.validate(config)).thenReturn(
new Config(
singletonList(new ConfigValue(key, null, Collections.emptyList(), errors))
List.of(new ConfigValue(key, null, List.of(), errors))
)
);
ConfigDef configDef = new ConfigDef();
@ -938,7 +934,7 @@ public class StandaloneHerderTest {
verify(statusBackingStore).put(new TaskStatus(new ConnectorTaskId(CONNECTOR_NAME, 0), AbstractStatus.State.DESTROYED, WORKER_ID, 0));
stopCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
herder.taskConfigs(CONNECTOR_NAME, taskConfigsCallback);
assertEquals(Collections.emptyList(), taskConfigsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
assertEquals(List.of(), taskConfigsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
// herder.stop() should stop any running connectors and tasks even if destroyConnector was not invoked
herder.stop();
@ -953,7 +949,7 @@ public class StandaloneHerderTest {
initialize(false);
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets("unknown-connector",
Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
Map.of(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
alterOffsetsCallback);
ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
assertInstanceOf(NotFoundException.class, e.getCause());
@ -972,20 +968,20 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState(
10,
null,
Collections.singletonMap(CONNECTOR_NAME, 3),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.PAUSED),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_NAME, 3),
Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Map.of(CONNECTOR_NAME, TargetState.PAUSED),
Map.of(),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Set.of(),
Set.of()
);
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets(CONNECTOR_NAME,
Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
Map.of(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
alterOffsetsCallback);
ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
assertInstanceOf(BadRequestException.class, e.getCause());
@ -1011,19 +1007,19 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState(
10,
null,
Collections.singletonMap(CONNECTOR_NAME, 0),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STOPPED),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_NAME, 0),
Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Map.of(CONNECTOR_NAME, TargetState.STOPPED),
Map.of(),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Set.of(),
Set.of()
);
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets(CONNECTOR_NAME,
Collections.singletonMap(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")),
Map.of(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
alterOffsetsCallback);
assertEquals(msg, alterOffsetsCallback.get(1000, TimeUnit.MILLISECONDS));
}
@ -1044,15 +1040,15 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState(
10,
null,
Collections.singletonMap(CONNECTOR_NAME, 0),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STOPPED),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(),
Collections.emptySet()
Map.of(CONNECTOR_NAME, 0),
Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Map.of(CONNECTOR_NAME, TargetState.STOPPED),
Map.of(),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Set.of(),
Set.of()
);
FutureCallback<Message> resetOffsetsCallback = new FutureCallback<>();
herder.resetConnectorOffsets(CONNECTOR_NAME, resetOffsetsCallback);
@ -1078,7 +1074,7 @@ public class StandaloneHerderTest {
assertEquals(createdInfo(SourceSink.SOURCE), connectorInfo.result());
// Prepare for task config update
when(worker.connectorNames()).thenReturn(Collections.singleton(CONNECTOR_NAME));
when(worker.connectorNames()).thenReturn(Set.of(CONNECTOR_NAME));
expectStop();
@ -1090,8 +1086,8 @@ public class StandaloneHerderTest {
updatedTaskConfig2.put("dummy-task-property", "2");
when(worker.connectorTaskConfigs(eq(CONNECTOR_NAME), any()))
.thenReturn(
Collections.singletonList(updatedTaskConfig1),
Collections.singletonList(updatedTaskConfig2));
List.of(updatedTaskConfig1),
List.of(updatedTaskConfig2));
// Set new config on the connector and tasks
FutureCallback<Herder.Created<ConnectorInfo>> reconfigureCallback = new FutureCallback<>();
@ -1102,7 +1098,7 @@ public class StandaloneHerderTest {
// Wait on connector update
Herder.Created<ConnectorInfo> updatedConnectorInfo = reconfigureCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
ConnectorInfo expectedConnectorInfo = new ConnectorInfo(CONNECTOR_NAME, newConfig, singletonList(new ConnectorTaskId(CONNECTOR_NAME, 0)), ConnectorType.SOURCE);
ConnectorInfo expectedConnectorInfo = new ConnectorInfo(CONNECTOR_NAME, newConfig, List.of(new ConnectorTaskId(CONNECTOR_NAME, 0)), ConnectorType.SOURCE);
assertEquals(expectedConnectorInfo, updatedConnectorInfo.result());
verify(statusBackingStore, times(2)).put(new TaskStatus(new ConnectorTaskId(CONNECTOR_NAME, 0), TaskStatus.State.DESTROYED, WORKER_ID, 0));
@ -1140,19 +1136,19 @@ public class StandaloneHerderTest {
Map<String, String> generatedTaskProps = taskConfig(sourceSink);
if (mockConnectorTaskConfigs) {
when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig)).thenReturn(singletonList(generatedTaskProps));
when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig)).thenReturn(List.of(generatedTaskProps));
}
ClusterConfigState configState = new ClusterConfigState(
-1,
null,
Collections.singletonMap(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps),
Collections.emptyMap(),
Collections.emptyMap(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Map.of(CONNECTOR_NAME, 1),
Map.of(CONNECTOR_NAME, connectorConfig),
Map.of(CONNECTOR_NAME, TargetState.STARTED),
Map.of(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps),
Map.of(),
Map.of(),
Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(),
new HashSet<>(),
transformer);
@ -1192,13 +1188,13 @@ public class StandaloneHerderTest {
private ConnectorInfo createdInfo(SourceSink sourceSink) {
return new ConnectorInfo(CONNECTOR_NAME, connectorConfig(sourceSink),
singletonList(new ConnectorTaskId(CONNECTOR_NAME, 0)),
List.of(new ConnectorTaskId(CONNECTOR_NAME, 0)),
SourceSink.SOURCE == sourceSink ? ConnectorType.SOURCE : ConnectorType.SINK);
}
private void expectStop() {
ConnectorTaskId task = new ConnectorTaskId(CONNECTOR_NAME, 0);
doNothing().when(worker).stopAndAwaitTasks(singletonList(task));
doNothing().when(worker).stopAndAwaitTasks(List.of(task));
doNothing().when(worker).stopAndAwaitConnector(CONNECTOR_NAME);
}
@ -1253,7 +1249,7 @@ public class StandaloneHerderTest {
// Set up validation for each config
for (Map<String, String> config : configs) {
when(connectorMock.validate(config)).thenReturn(new Config(Collections.emptyList()));
when(connectorMock.validate(config)).thenReturn(new Config(List.of()));
}
}

View File

@ -40,8 +40,8 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
@ -440,7 +440,7 @@ public class ConnectorOffsetBackingStoreTest {
MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(AutoOffsetResetStrategy.LATEST.name());
Node noNode = Node.noNode();
Node[] nodes = new Node[]{noNode};
consumer.updatePartitions(topic, Collections.singletonList(new PartitionInfo(topic, 0, noNode, nodes, nodes)));
consumer.updatePartitions(topic, List.of(new PartitionInfo(topic, 0, noNode, nodes, nodes)));
consumer.updateBeginningOffsets(mkMap(mkEntry(new TopicPartition(topic, 0), 100L)));
return consumer;
}

View File

@ -34,8 +34,6 @@ import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.file.Files;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -79,7 +77,7 @@ public class FileOffsetBackingStoreTest {
converter = mock(Converter.class);
// This is only needed for storing deserialized connector partitions, which we don't test in most of the cases here
when(converter.toConnectData(anyString(), any(byte[].class))).thenReturn(new SchemaAndValue(null,
Arrays.asList("connector", Collections.singletonMap("partitionKey", "dummy"))));
List.of("connector", Map.of("partitionKey", "dummy"))));
store = new FileOffsetBackingStore(converter);
tempFile = assertDoesNotThrow(() -> File.createTempFile("fileoffsetbackingstore", null));
Map<String, String> props = new HashMap<>();
@ -105,7 +103,7 @@ public class FileOffsetBackingStoreTest {
store.set(FIRST_SET, setCallback).get();
Map<ByteBuffer, ByteBuffer> values = store.get(Arrays.asList(buffer("key"), buffer("bad"))).get();
Map<ByteBuffer, ByteBuffer> values = store.get(List.of(buffer("key"), buffer("bad"))).get();
assertEquals(buffer("value"), values.get(buffer("key")));
assertNull(values.get(buffer("bad")));
verify(setCallback).onCompletion(isNull(), isNull());
@ -123,7 +121,7 @@ public class FileOffsetBackingStoreTest {
FileOffsetBackingStore restore = new FileOffsetBackingStore(converter);
restore.configure(config);
restore.start();
Map<ByteBuffer, ByteBuffer> values = restore.get(Collections.singletonList(buffer("key"))).get();
Map<ByteBuffer, ByteBuffer> values = restore.get(List.of(buffer("key"))).get();
assertEquals(buffer("value"), values.get(buffer("key")));
verify(setCallback).onCompletion(isNull(), isNull());
}
@ -135,26 +133,26 @@ public class FileOffsetBackingStoreTest {
// This test actually requires the offset store to track deserialized source partitions, so we can't use the member variable mock converter
JsonConverter jsonConverter = new JsonConverter();
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
jsonConverter.configure(Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
Map<ByteBuffer, ByteBuffer> serializedPartitionOffsets = new HashMap<>();
serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue"))
serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
);
store.set(serializedPartitionOffsets, setCallback).get();
serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue2"))
serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
serialize(jsonConverter, Map.of("offsetKey", "offsetValue2"))
);
serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue2")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue"))
serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue2")),
serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
);
serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector2", Collections.singletonMap("partitionKey", "partitionValue")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue"))
serializeKey(jsonConverter, "connector2", Map.of("partitionKey", "partitionValue")),
serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
);
store.set(serializedPartitionOffsets, setCallback).get();
@ -167,23 +165,23 @@ public class FileOffsetBackingStoreTest {
Set<Map<String, Object>> connectorPartitions1 = restore.connectorPartitions("connector1");
Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue2"));
expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue2"));
assertEquals(expectedConnectorPartition1, connectorPartitions1);
Set<Map<String, Object>> connectorPartitions2 = restore.connectorPartitions("connector2");
Set<Map<String, Object>> expectedConnectorPartition2 = Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue"));
Set<Map<String, Object>> expectedConnectorPartition2 = Set.of(Map.of("partitionKey", "partitionValue"));
assertEquals(expectedConnectorPartition2, connectorPartitions2);
serializedPartitionOffsets.clear();
// Null valued offset for a partition key should remove that partition for the connector
serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")),
serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
null
);
restore.set(serializedPartitionOffsets, setCallback).get();
connectorPartitions1 = restore.connectorPartitions("connector1");
assertEquals(Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue2")), connectorPartitions1);
assertEquals(Set.of(Map.of("partitionKey", "partitionValue2")), connectorPartitions1);
verify(setCallback, times(3)).onCompletion(isNull(), isNull());
}
@ -193,7 +191,7 @@ public class FileOffsetBackingStoreTest {
}
private static ByteBuffer serializeKey(Converter converter, String connectorName, Map<String, Object> sourcePartition) {
List<Object> nameAndPartition = Arrays.asList(connectorName, sourcePartition);
List<Object> nameAndPartition = List.of(connectorName, sourcePartition);
return serialize(converter, nameAndPartition);
}

View File

@ -61,8 +61,6 @@ import org.mockito.stubbing.Answer;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
@ -126,41 +124,41 @@ public class KafkaConfigBackingStoreTest {
DEFAULT_CONFIG_STORAGE_PROPS.put(DistributedConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
}
private static final List<String> CONNECTOR_IDS = Arrays.asList("connector1", "connector2");
private static final List<String> CONNECTOR_CONFIG_KEYS = Arrays.asList("connector-connector1", "connector-connector2");
private static final List<String> COMMIT_TASKS_CONFIG_KEYS = Arrays.asList("commit-connector1", "commit-connector2");
private static final List<String> CONNECTOR_IDS = List.of("connector1", "connector2");
private static final List<String> CONNECTOR_CONFIG_KEYS = List.of("connector-connector1", "connector-connector2");
private static final List<String> COMMIT_TASKS_CONFIG_KEYS = List.of("commit-connector1", "commit-connector2");
private static final List<String> TARGET_STATE_KEYS = Arrays.asList("target-state-connector1", "target-state-connector2");
private static final List<String> CONNECTOR_TASK_COUNT_RECORD_KEYS = Arrays.asList("tasks-fencing-connector1", "tasks-fencing-connector2");
private static final List<String> TARGET_STATE_KEYS = List.of("target-state-connector1", "target-state-connector2");
private static final List<String> CONNECTOR_TASK_COUNT_RECORD_KEYS = List.of("tasks-fencing-connector1", "tasks-fencing-connector2");
private static final String CONNECTOR_1_NAME = "connector1";
private static final String CONNECTOR_2_NAME = "connector2";
private static final List<String> RESTART_CONNECTOR_KEYS = Arrays.asList(RESTART_KEY(CONNECTOR_1_NAME), RESTART_KEY(CONNECTOR_2_NAME));
private static final List<String> RESTART_CONNECTOR_KEYS = List.of(RESTART_KEY(CONNECTOR_1_NAME), RESTART_KEY(CONNECTOR_2_NAME));
// Need a) connector with multiple tasks and b) multiple connectors
private static final List<ConnectorTaskId> TASK_IDS = Arrays.asList(
private static final List<ConnectorTaskId> TASK_IDS = List.of(
new ConnectorTaskId("connector1", 0),
new ConnectorTaskId("connector1", 1),
new ConnectorTaskId("connector2", 0)
);
private static final List<String> TASK_CONFIG_KEYS = Arrays.asList("task-connector1-0", "task-connector1-1", "task-connector2-0");
private static final List<String> TASK_CONFIG_KEYS = List.of("task-connector1-0", "task-connector1-1", "task-connector2-0");
// Need some placeholders -- the contents don't matter here, just that they are restored properly
private static final List<Map<String, String>> SAMPLE_CONFIGS = Arrays.asList(
Collections.singletonMap("config-key-one", "config-value-one"),
Collections.singletonMap("config-key-two", "config-value-two"),
Collections.singletonMap("config-key-three", "config-value-three")
private static final List<Map<String, String>> SAMPLE_CONFIGS = List.of(
Map.of("config-key-one", "config-value-one"),
Map.of("config-key-two", "config-value-two"),
Map.of("config-key-three", "config-value-three")
);
private static final List<Struct> TASK_CONFIG_STRUCTS = Arrays.asList(
private static final List<Struct> TASK_CONFIG_STRUCTS = List.of(
new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1))
);
private static final Struct ONLY_FAILED_MISSING_STRUCT = new Struct(KafkaConfigBackingStore.RESTART_REQUEST_V0).put(INCLUDE_TASKS_FIELD_NAME, false);
private static final Struct INCLUDE_TASKS_MISSING_STRUCT = new Struct(KafkaConfigBackingStore.RESTART_REQUEST_V0).put(ONLY_FAILED_FIELD_NAME, true);
private static final List<Struct> RESTART_REQUEST_STRUCTS = Arrays.asList(
private static final List<Struct> RESTART_REQUEST_STRUCTS = List.of(
new Struct(KafkaConfigBackingStore.RESTART_REQUEST_V0).put(ONLY_FAILED_FIELD_NAME, true).put(INCLUDE_TASKS_FIELD_NAME, false),
ONLY_FAILED_MISSING_STRUCT,
INCLUDE_TASKS_MISSING_STRUCT);
private static final List<Struct> CONNECTOR_CONFIG_STRUCTS = Arrays.asList(
private static final List<Struct> CONNECTOR_CONFIG_STRUCTS = List.of(
new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)),
new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)),
new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2))
@ -173,14 +171,14 @@ public class KafkaConfigBackingStoreTest {
private static final Struct TARGET_STATE_STOPPED = new Struct(KafkaConfigBackingStore.TARGET_STATE_V1)
.put("state", "PAUSED")
.put("state.v2", "STOPPED");
private static final List<Struct> CONNECTOR_TASK_COUNT_RECORD_STRUCTS = Arrays.asList(
private static final List<Struct> CONNECTOR_TASK_COUNT_RECORD_STRUCTS = List.of(
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 6),
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9),
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 2)
);
// The exact format doesn't matter here since both conversions are mocked
private static final List<byte[]> CONFIGS_SERIALIZED = Arrays.asList(
private static final List<byte[]> CONFIGS_SERIALIZED = List.of(
"config-bytes-1".getBytes(), "config-bytes-2".getBytes(), "config-bytes-3".getBytes(),
"config-bytes-4".getBytes(), "config-bytes-5".getBytes(), "config-bytes-6".getBytes(),
"config-bytes-7".getBytes(), "config-bytes-8".getBytes(), "config-bytes-9".getBytes()
@ -191,7 +189,7 @@ public class KafkaConfigBackingStoreTest {
private static final Struct TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR
= new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 0);
private static final List<byte[]> TARGET_STATES_SERIALIZED = Arrays.asList(
private static final List<byte[]> TARGET_STATES_SERIALIZED = List.of(
"started".getBytes(), "paused".getBytes(), "stopped".getBytes()
);
@Mock
@ -308,8 +306,8 @@ public class KafkaConfigBackingStoreTest {
String configKey = CONNECTOR_CONFIG_KEYS.get(1);
String targetStateKey = TARGET_STATE_KEYS.get(1);
doAnswer(expectReadToEnd(Collections.singletonMap(CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
.doAnswer(expectReadToEnd(Collections.singletonMap(CONNECTOR_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(1))))
doAnswer(expectReadToEnd(Map.of(CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
.doAnswer(expectReadToEnd(Map.of(CONNECTOR_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(1))))
// Config deletion
.doAnswer(expectReadToEnd(new LinkedHashMap<>() {{
put(configKey, null);
@ -426,7 +424,7 @@ public class KafkaConfigBackingStoreTest {
assertEquals(-1, configState.offset());
assertEquals(0, configState.connectors().size());
Exception thrownException = new ExecutionException(new TopicAuthorizationException(Collections.singleton("test")));
Exception thrownException = new ExecutionException(new TopicAuthorizationException(Set.of("test")));
when(producerFuture.get(anyLong(), any(TimeUnit.class))).thenThrow(thrownException);
// verify that the producer exception from KafkaBasedLog::send is propagated
@ -510,8 +508,8 @@ public class KafkaConfigBackingStoreTest {
doReturn(fencableProducer).when(configStorage).createFencableProducer();
// And write the task count record successfully
when(fencableProducer.send(any(ProducerRecord.class))).thenReturn(null);
doAnswer(expectReadToEnd(Collections.singletonMap(CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
.doAnswer(expectReadToEnd(Collections.singletonMap(CONNECTOR_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2))))
doAnswer(expectReadToEnd(Map.of(CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
.doAnswer(expectReadToEnd(Map.of(CONNECTOR_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2))))
.when(configLog).readToEnd();
when(converter.toConnectData(TOPIC, CONFIGS_SERIALIZED.get(0)))
.thenReturn(new SchemaAndValue(null, structToMap(CONNECTOR_TASK_COUNT_RECORD_STRUCTS.get(0))));
@ -570,7 +568,7 @@ public class KafkaConfigBackingStoreTest {
@Test
public void testRestoreTargetStateUnexpectedDeletion() {
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -599,7 +597,7 @@ public class KafkaConfigBackingStoreTest {
// The target state deletion should reset the state to STARTED
ClusterConfigState configState = configStorage.snapshot();
assertEquals(5, configState.offset()); // Should always be next to be read, even if uncommitted
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
configStorage.stop();
@ -608,7 +606,7 @@ public class KafkaConfigBackingStoreTest {
@Test
public void testRestoreTargetState() {
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -643,7 +641,7 @@ public class KafkaConfigBackingStoreTest {
// Should see a single connector with initial state paused
ClusterConfigState configState = configStorage.snapshot();
assertEquals(6, configState.offset()); // Should always be next to be read, even if uncommitted
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(TargetState.PAUSED, configState.targetState(CONNECTOR_IDS.get(0)));
assertEquals(TargetState.STOPPED, configState.targetState(CONNECTOR_IDS.get(1)));
@ -657,7 +655,7 @@ public class KafkaConfigBackingStoreTest {
// that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_TASK_COUNT_RECORD_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
@ -699,18 +697,18 @@ public class KafkaConfigBackingStoreTest {
// Should see a single connector and its config should be the last one seen anywhere in the log
ClusterConfigState configState = configStorage.snapshot();
assertEquals(logOffset, configState.offset()); // Should always be next to be read, even if uncommitted
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
// CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0)));
// Should see 2 tasks for that connector. Only config updates before the root key update should be reflected
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(List.of(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(CONNECTOR_IDS.get(0)));
// Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(9, (int) configState.taskCountRecord(CONNECTOR_IDS.get(1)));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(Collections.singleton("connector1"), configState.connectorsPendingFencing);
assertEquals(Set.of(), configState.inconsistentConnectors());
assertEquals(Set.of("connector1"), configState.connectorsPendingFencing);
// Shouldn't see any callbacks since this is during startup
configStorage.stop();
@ -723,7 +721,7 @@ public class KafkaConfigBackingStoreTest {
// that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -769,7 +767,7 @@ public class KafkaConfigBackingStoreTest {
// that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -807,13 +805,13 @@ public class KafkaConfigBackingStoreTest {
// Should see a single connector and its config should be the last one seen anywhere in the log
ClusterConfigState configState = configStorage.snapshot();
assertEquals(8, configState.offset()); // Should always be next to be read, even if uncommitted
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
// CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0)));
// Should see 0 tasks for that connector.
assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(List.of(), configState.tasks(CONNECTOR_IDS.get(0)));
// Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(Set.of(), configState.inconsistentConnectors());
// Shouldn't see any callbacks since this is during startup
configStorage.stop();
@ -1020,7 +1018,7 @@ public class KafkaConfigBackingStoreTest {
@Test
public void testBackgroundConnectorDeletion() throws Exception {
// verify that we handle connector deletions correctly when they come up through the log
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -1072,7 +1070,7 @@ public class KafkaConfigBackingStoreTest {
assertEquals(0, configState.taskCount(CONNECTOR_IDS.get(0)));
// Ensure that the deleted connector's deferred task updates have been cleaned up
// in order to prevent unbounded growth of the map
assertEquals(Collections.emptyMap(), configStorage.deferredTaskUpdates);
assertEquals(Map.of(), configStorage.deferredTaskUpdates);
configStorage.stop();
verify(configLog).stop();
@ -1083,7 +1081,7 @@ public class KafkaConfigBackingStoreTest {
// Test a case where a failure and compaction has left us in an inconsistent state when reading the log.
// We start out by loading an initial configuration where we started to write a task update, and then
// compaction cleaned up the earlier record.
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
// This is the record that has been compacted:
@ -1110,13 +1108,13 @@ public class KafkaConfigBackingStoreTest {
// After reading the log, it should have been in an inconsistent state
ClusterConfigState configState = configStorage.snapshot();
assertEquals(6, configState.offset()); // Should always be next to be read, not last committed
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
// Inconsistent data should leave us with no tasks listed for the connector and an entry in the inconsistent list
assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(List.of(), configState.tasks(CONNECTOR_IDS.get(0)));
// Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertNull(configState.taskConfig(TASK_IDS.get(0)));
assertNull(configState.taskConfig(TASK_IDS.get(1)));
assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configState.inconsistentConnectors());
assertEquals(Set.of(CONNECTOR_IDS.get(0)), configState.inconsistentConnectors());
// Records to be read by consumer as it reads to the end of the log
LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>();
@ -1136,20 +1134,20 @@ public class KafkaConfigBackingStoreTest {
// Next, issue a write that has everything that is needed and it should be accepted. Note that in this case
// we are going to shrink the number of tasks to 1
configStorage.putTaskConfigs("connector1", Collections.singletonList(SAMPLE_CONFIGS.get(0)));
configStorage.putTaskConfigs("connector1", List.of(SAMPLE_CONFIGS.get(0)));
// Validate updated config
configState = configStorage.snapshot();
// This is only two more ahead of the last one because multiple calls fail, and so their configs are not written
// to the topic. Only the last call with 1 task config + 1 commit actually gets written.
assertEquals(8, configState.offset());
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(Collections.singletonList(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(List.of(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(Set.of(), configState.inconsistentConnectors());
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
verify(configUpdateListener).onTaskConfigUpdate(Collections.singletonList(TASK_IDS.get(0)));
verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(0)));
configStorage.stop();
verify(configLog).stop();
@ -1168,7 +1166,7 @@ public class KafkaConfigBackingStoreTest {
}
private void testPutRestartRequest(RestartRequest restartRequest) {
expectStart(Collections.emptyList(), Collections.emptyMap());
expectStart(List.of(), Map.of());
when(configLog.partitionCount()).thenReturn(1);
configStorage.setupAndCreateKafkaBasedLog(TOPIC, config);
@ -1204,7 +1202,7 @@ public class KafkaConfigBackingStoreTest {
// Restoring data should notify only of the latest values after loading is complete. This also validates
// that inconsistent state doesn't prevent startup.
// Overwrite each type at least once to ensure we see the latest data after loading
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(1),
@ -1250,7 +1248,7 @@ public class KafkaConfigBackingStoreTest {
// Records to be read by consumer as it reads to the end of the log
doAnswer(expectReadToEnd(new LinkedHashMap<>())).
doAnswer(expectReadToEnd(Collections.singletonMap(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
doAnswer(expectReadToEnd(Map.of(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0))))
.when(configLog).readToEnd();
expectConvertWriteRead(
@ -1258,7 +1256,7 @@ public class KafkaConfigBackingStoreTest {
"tasks", 0); // We have 0 tasks
// Bootstrap as if we had already added the connector, but no tasks had been added yet
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList());
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), List.of());
// Null before writing
@ -1267,19 +1265,19 @@ public class KafkaConfigBackingStoreTest {
// Writing task configs should block until all the writes have been performed and the root record update
// has completed
List<Map<String, String>> taskConfigs = Collections.emptyList();
List<Map<String, String>> taskConfigs = List.of();
configStorage.putTaskConfigs("connector1", taskConfigs);
// Validate root config by listing all connectors and tasks
configState = configStorage.snapshot();
assertEquals(1, configState.offset());
String connectorName = CONNECTOR_IDS.get(0);
assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(Collections.emptyList(), configState.tasks(connectorName));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(List.of(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(List.of(), configState.tasks(connectorName));
assertEquals(Set.of(), configState.inconsistentConnectors());
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
verify(configUpdateListener).onTaskConfigUpdate(Collections.emptyList());
verify(configUpdateListener).onTaskConfigUpdate(List.of());
configStorage.stop();
verify(configLog).stop();
@ -1288,7 +1286,7 @@ public class KafkaConfigBackingStoreTest {
@Test
public void testBackgroundUpdateTargetState() throws Exception {
// verify that we handle target state changes correctly when they come up through the log
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -1314,7 +1312,7 @@ public class KafkaConfigBackingStoreTest {
// Should see a single connector with initial state started
ClusterConfigState configState = configStorage.snapshot();
assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configStorage.connectorTargetStates.keySet());
assertEquals(Set.of(CONNECTOR_IDS.get(0)), configStorage.connectorTargetStates.keySet());
assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
LinkedHashMap<String, byte[]> serializedAfterStartup = new LinkedHashMap<>();
@ -1343,7 +1341,7 @@ public class KafkaConfigBackingStoreTest {
@Test
public void testSameTargetState() {
// verify that we handle target state changes correctly when they come up through the log
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0),
@ -1395,7 +1393,7 @@ public class KafkaConfigBackingStoreTest {
// Pre-populate the config topic with a couple of logger level records; these should be ignored (i.e.,
// not reported to the update listener)
List<ConsumerRecord<String, byte[]>> existingRecords = Arrays.asList(
List<ConsumerRecord<String, byte[]>> existingRecords = List.of(
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, "logger-cluster-" + logger1,
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()
),
@ -1482,7 +1480,7 @@ public class KafkaConfigBackingStoreTest {
CONNECTOR_TASK_COUNT_RECORD_KEYS.get(0), KafkaConfigBackingStore.TASK_COUNT_RECORD_V0, CONFIGS_SERIALIZED.get(3),
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 4));
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList());
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), List.of());
// Before anything is written
String connectorName = CONNECTOR_IDS.get(0);
@ -1493,7 +1491,7 @@ public class KafkaConfigBackingStoreTest {
// Writing task configs should block until all the writes have been performed and the root record update
// has completed
List<Map<String, String>> taskConfigs = Arrays.asList(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
List<Map<String, String>> taskConfigs = List.of(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
configStorage.putTaskConfigs("connector1", taskConfigs);
configState = configStorage.snapshot();
@ -1511,7 +1509,7 @@ public class KafkaConfigBackingStoreTest {
assertEquals(0, (long) configState.taskConfigGeneration(connectorName));
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
verify(configUpdateListener).onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(0), TASK_IDS.get(1)));
configStorage.stop();
verify(configLog).stop();
@ -1545,7 +1543,7 @@ public class KafkaConfigBackingStoreTest {
COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2),
new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2)); // Starts with 0 tasks, after update has 2
// Bootstrap as if we had already added the connector, but no tasks had been added yet
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList());
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), List.of());
// Null before writing
ClusterConfigState configState = configStorage.snapshot();
@ -1555,21 +1553,21 @@ public class KafkaConfigBackingStoreTest {
// Writing task configs should block until all the writes have been performed and the root record update
// has completed
List<Map<String, String>> taskConfigs = Arrays.asList(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
List<Map<String, String>> taskConfigs = List.of(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
configStorage.putTaskConfigs("connector1", taskConfigs);
// Validate root config by listing all connectors and tasks
configState = configStorage.snapshot();
assertEquals(3, configState.offset());
String connectorName = CONNECTOR_IDS.get(0);
assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName));
assertEquals(List.of(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(List.of(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(Set.of(), configState.inconsistentConnectors());
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
verify(configUpdateListener).onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(0), TASK_IDS.get(1)));
configStorage.stop();
verify(configLog).stop();
@ -1616,8 +1614,8 @@ public class KafkaConfigBackingStoreTest {
assertNull(configState.taskConfig(TASK_IDS.get(1)));
// Bootstrap as if we had already added the connector, but no tasks had been added yet
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.emptyList());
List<Map<String, String>> taskConfigs = Arrays.asList(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
addConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), List.of());
List<Map<String, String>> taskConfigs = List.of(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1));
configStorage.putTaskConfigs("connector1", taskConfigs);
expectConvertWriteRead2(
@ -1627,8 +1625,8 @@ public class KafkaConfigBackingStoreTest {
COMMIT_TASKS_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(4),
new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 1)); // Starts with 2 tasks, after update has 3
addConnector(CONNECTOR_IDS.get(1), SAMPLE_CONFIGS.get(1), Collections.emptyList());
taskConfigs = Collections.singletonList(SAMPLE_CONFIGS.get(2));
addConnector(CONNECTOR_IDS.get(1), SAMPLE_CONFIGS.get(1), List.of());
taskConfigs = List.of(SAMPLE_CONFIGS.get(2));
configStorage.putTaskConfigs("connector2", taskConfigs);
// Validate root config by listing all connectors and tasks
@ -1636,17 +1634,17 @@ public class KafkaConfigBackingStoreTest {
assertEquals(5, configState.offset());
String connectorName1 = CONNECTOR_IDS.get(0);
String connectorName2 = CONNECTOR_IDS.get(1);
assertEquals(Arrays.asList(connectorName1, connectorName2), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName1));
assertEquals(Collections.singletonList(TASK_IDS.get(2)), configState.tasks(connectorName2));
assertEquals(List.of(connectorName1, connectorName2), new ArrayList<>(configState.connectors()));
assertEquals(List.of(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName1));
assertEquals(List.of(TASK_IDS.get(2)), configState.tasks(connectorName2));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(SAMPLE_CONFIGS.get(2), configState.taskConfig(TASK_IDS.get(2)));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors());
assertEquals(Set.of(), configState.inconsistentConnectors());
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks
verify(configUpdateListener).onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)));
verify(configUpdateListener).onTaskConfigUpdate(Collections.singletonList(TASK_IDS.get(2)));
verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(0), TASK_IDS.get(1)));
verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(2)));
configStorage.stop();
verify(configLog).stop();

View File

@ -45,9 +45,9 @@ import org.mockito.quality.Strictness;
import java.nio.ByteBuffer;
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.Optional;
import java.util.Set;
@ -145,7 +145,7 @@ public class KafkaOffsetBackingStoreTest {
if (mockKeyConverter) {
when(keyConverter.toConnectData(any(), any())).thenReturn(new SchemaAndValue(null,
Arrays.asList("connector", Collections.singletonMap("partitionKey", "dummy"))));
List.of("connector", Map.of("partitionKey", "dummy"))));
}
store = spy(new KafkaOffsetBackingStore(adminSupplier, clientIdBase, keyConverter));
@ -233,7 +233,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting from empty store should return nulls
Map<ByteBuffer, ByteBuffer> offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
Map<ByteBuffer, ByteBuffer> offsets = store.get(List.of(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
// Since we didn't read them yet, these will be null
assertNull(offsets.get(TP0_KEY));
assertNull(offsets.get(TP1_KEY));
@ -270,7 +270,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it
offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
offsets = store.get(List.of(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
assertEquals(TP0_VALUE, offsets.get(TP0_KEY));
assertEquals(TP1_VALUE, offsets.get(TP1_KEY));
@ -287,7 +287,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it
offsets = store.get(Arrays.asList(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
offsets = store.get(List.of(TP0_KEY, TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
assertEquals(TP0_VALUE_NEW, offsets.get(TP0_KEY));
assertEquals(TP1_VALUE_NEW, offsets.get(TP1_KEY));
@ -363,7 +363,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it
offsets = store.get(Collections.singletonList(TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
offsets = store.get(List.of(TP1_KEY)).get(10000, TimeUnit.MILLISECONDS);
assertNull(offsets.get(TP1_KEY));
// Just verifying that KafkaOffsetBackingStore::get returns null isn't enough, we also need to verify that the mapping for the source partition key is removed.
@ -488,7 +488,7 @@ public class KafkaOffsetBackingStoreTest {
@Test
public void testConnectorPartitions() throws Exception {
JsonConverter jsonConverter = new JsonConverter();
jsonConverter.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
jsonConverter.configure(Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
store = spy(new KafkaOffsetBackingStore(() -> {
fail("Should not attempt to instantiate admin in these tests");
return null;
@ -506,57 +506,57 @@ public class KafkaOffsetBackingStoreTest {
doAnswer(invocation -> {
capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), TP0_VALUE.array(),
jsonConverter.fromConnectData("", null, List.of("connector1",
Map.of("partitionKey", "partitionValue1"))), TP0_VALUE.array(),
new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), TP1_VALUE.array(),
jsonConverter.fromConnectData("", null, List.of("connector1",
Map.of("partitionKey", "partitionValue1"))), TP1_VALUE.array(),
new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1",
Collections.singletonMap("partitionKey", "partitionValue2"))), TP2_VALUE.array(),
jsonConverter.fromConnectData("", null, List.of("connector1",
Map.of("partitionKey", "partitionValue2"))), TP2_VALUE.array(),
new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector2",
Collections.singletonMap("partitionKey", "partitionValue"))), TP1_VALUE.array(),
jsonConverter.fromConnectData("", null, List.of("connector2",
Map.of("partitionKey", "partitionValue"))), TP1_VALUE.array(),
new RecordHeaders(), Optional.empty()));
storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null);
return null;
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Trigger a read to the end of the log
store.get(Collections.emptyList()).get(10000, TimeUnit.MILLISECONDS);
store.get(List.of()).get(10000, TimeUnit.MILLISECONDS);
Set<Map<String, Object>> connectorPartitions1 = store.connectorPartitions("connector1");
Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue2"));
expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue2"));
assertEquals(expectedConnectorPartition1, connectorPartitions1);
Set<Map<String, Object>> connectorPartitions2 = store.connectorPartitions("connector2");
Set<Map<String, Object>> expectedConnectorPartition2 = Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue"));
Set<Map<String, Object>> expectedConnectorPartition2 = Set.of(Map.of("partitionKey", "partitionValue"));
assertEquals(expectedConnectorPartition2, connectorPartitions2);
doAnswer(invocation -> {
capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), null,
jsonConverter.fromConnectData("", null, List.of("connector1",
Map.of("partitionKey", "partitionValue1"))), null,
new RecordHeaders(), Optional.empty()));
storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null);
return null;
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Trigger a read to the end of the log
store.get(Collections.emptyList()).get(10000, TimeUnit.MILLISECONDS);
store.get(List.of()).get(10000, TimeUnit.MILLISECONDS);
// Null valued offset for a partition key should remove that partition for the connector
connectorPartitions1 = store.connectorPartitions("connector1");
assertEquals(Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue2")), connectorPartitions1);
assertEquals(Set.of(Map.of("partitionKey", "partitionValue2")), connectorPartitions1);
store.stop();
verify(storeLog).stop();

View File

@ -35,8 +35,8 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
@ -75,7 +75,7 @@ public class KafkaStatusBackingStoreFormatTest {
public void setup() {
time = new MockTime();
JsonConverter converter = new JsonConverter();
converter.configure(Collections.singletonMap(SCHEMAS_ENABLE_CONFIG, false), false);
converter.configure(Map.of(SCHEMAS_ENABLE_CONFIG, false), false);
store = new KafkaStatusBackingStore(new MockTime(), converter, STATUS_TOPIC, () -> null, kafkaBasedLog);
}
@ -182,7 +182,7 @@ public class KafkaStatusBackingStoreFormatTest {
store.read(statusRecord);
assertTrue(store.topics.containsKey("bar"));
assertFalse(store.topics.get("bar").containsKey("foo"));
assertEquals(Collections.emptyMap(), store.topics.get("bar"));
assertEquals(Map.of(), store.topics.get("bar"));
}
@Test
@ -204,7 +204,7 @@ public class KafkaStatusBackingStoreFormatTest {
ConsumerRecord<String, byte[]> statusRecord = new ConsumerRecord<>(STATUS_TOPIC, 0, 0, key, valueCaptor.getValue());
store.read(statusRecord);
assertEquals(topicStatus, store.getTopic(FOO_CONNECTOR, FOO_TOPIC));
assertEquals(Collections.singleton(topicStatus), new HashSet<>(store.getAllTopics(FOO_CONNECTOR)));
assertEquals(Set.of(topicStatus), new HashSet<>(store.getAllTopics(FOO_CONNECTOR)));
}
@Test

View File

@ -28,9 +28,7 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
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;
@ -51,13 +49,13 @@ import static org.mockito.Mockito.verify;
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class MemoryConfigBackingStoreTest {
private static final List<String> CONNECTOR_IDS = Arrays.asList("connector1", "connector2");
private static final List<String> CONNECTOR_IDS = List.of("connector1", "connector2");
// Actual values are irrelevant here and can be used as either connector or task configurations
private static final List<Map<String, String>> SAMPLE_CONFIGS = Arrays.asList(
Collections.singletonMap("config-key-one", "config-value-one"),
Collections.singletonMap("config-key-two", "config-value-two"),
Collections.singletonMap("config-key-three", "config-value-three")
private static final List<Map<String, String>> SAMPLE_CONFIGS = List.of(
Map.of("config-key-one", "config-value-one"),
Map.of("config-key-two", "config-value-two"),
Map.of("config-key-three", "config-value-three")
);
@Mock
@ -142,10 +140,10 @@ public class MemoryConfigBackingStoreTest {
public void testPutTaskConfigs() {
// Can't write task configs for non-existent connector
assertThrows(IllegalArgumentException.class,
() -> configStore.putTaskConfigs(CONNECTOR_IDS.get(0), Collections.singletonList(SAMPLE_CONFIGS.get(1))));
() -> configStore.putTaskConfigs(CONNECTOR_IDS.get(0), List.of(SAMPLE_CONFIGS.get(1))));
configStore.putConnectorConfig(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), null);
configStore.putTaskConfigs(CONNECTOR_IDS.get(0), Collections.singletonList(SAMPLE_CONFIGS.get(1)));
configStore.putTaskConfigs(CONNECTOR_IDS.get(0), List.of(SAMPLE_CONFIGS.get(1)));
ClusterConfigState configState = configStore.snapshot();
ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_IDS.get(0), 0);
@ -153,7 +151,7 @@ public class MemoryConfigBackingStoreTest {
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(taskId));
verify(configUpdateListener).onConnectorConfigUpdate(eq(CONNECTOR_IDS.get(0)));
verify(configUpdateListener).onTaskConfigUpdate(eq(Collections.singleton(taskId)));
verify(configUpdateListener).onTaskConfigUpdate(eq(Set.of(taskId)));
}
@Test
@ -172,18 +170,18 @@ public class MemoryConfigBackingStoreTest {
}).when(configUpdateListener).onTaskConfigUpdate(anySet());
configStore.putConnectorConfig(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), null);
configStore.putTaskConfigs(CONNECTOR_IDS.get(0), Collections.singletonList(SAMPLE_CONFIGS.get(1)));
configStore.putTaskConfigs(CONNECTOR_IDS.get(0), List.of(SAMPLE_CONFIGS.get(1)));
configStore.removeTaskConfigs(CONNECTOR_IDS.get(0));
ClusterConfigState configState = configStore.snapshot();
assertEquals(0, configState.taskCount(CONNECTOR_IDS.get(0)));
assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0)));
assertEquals(List.of(), configState.tasks(CONNECTOR_IDS.get(0)));
verify(configUpdateListener).onConnectorConfigUpdate(eq(CONNECTOR_IDS.get(0)));
verify(configUpdateListener, times(2)).onTaskConfigUpdate(anySet());
ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_IDS.get(0), 0);
assertEquals(Arrays.asList(Collections.singleton(taskId), Collections.singleton(taskId)), onTaskConfigUpdateCaptures);
assertEquals(List.of(Set.of(taskId), Set.of(taskId)), onTaskConfigUpdateCaptures);
}
@Test

View File

@ -22,7 +22,7 @@ import org.apache.kafka.connect.util.ConnectorTaskId;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.Set;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
@ -44,7 +44,7 @@ public class MemoryStatusBackingStoreTest {
TaskStatus status = new TaskStatus(taskId, ConnectorStatus.State.RUNNING, "localhost:8083", 0);
store.put(status);
assertEquals(status, store.get(taskId));
assertEquals(Collections.singleton(status), store.getAll("connector"));
assertEquals(Set.of(status), store.getAll("connector"));
}
@Test
@ -63,5 +63,4 @@ public class MemoryStatusBackingStoreTest {
store.put(new TaskStatus(taskId, ConnectorStatus.State.DESTROYED, "localhost:8083", 0));
assertNull(store.get(taskId));
}
}

View File

@ -54,8 +54,8 @@ import static org.mockito.Mockito.when;
public class OffsetStorageWriterTest {
private static final String NAMESPACE = "namespace";
// Connect format - any types should be accepted here
private static final Map<String, Object> OFFSET_KEY = Collections.singletonMap("key", "key");
private static final Map<String, Object> OFFSET_VALUE = Collections.singletonMap("key", 12);
private static final Map<String, Object> OFFSET_KEY = Map.of("key", "key");
private static final Map<String, Object> OFFSET_VALUE = Map.of("key", 12);
// Serialized
private static final byte[] OFFSET_KEY_SERIALIZED = "key-serialized".getBytes();

View File

@ -26,8 +26,8 @@ import org.junit.jupiter.api.Test;
import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
@ -40,7 +40,7 @@ public class OffsetUtilsTest {
private static final JsonConverter CONVERTER = new JsonConverter();
static {
CONVERTER.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
CONVERTER.configure(Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
}
@Test
@ -60,18 +60,18 @@ public class OffsetUtilsTest {
@Test
public void testValidateFormatMapWithNonPrimitiveKeys() {
Map<Object, Object> offsetData = Collections.singletonMap("key", new Object());
Map<Object, Object> offsetData = Map.of("key", new Object());
DataException e = assertThrows(DataException.class, () -> OffsetUtils.validateFormat(offsetData));
assertTrue(e.getMessage().contains("Offsets may only contain primitive types as values"));
Map<Object, Object> offsetData2 = Collections.singletonMap("key", new ArrayList<>());
Map<Object, Object> offsetData2 = Map.of("key", new ArrayList<>());
e = assertThrows(DataException.class, () -> OffsetUtils.validateFormat(offsetData2));
assertTrue(e.getMessage().contains("Offsets may only contain primitive types as values"));
}
@Test
public void testValidateFormatWithValidFormat() {
Map<Object, Object> offsetData = Collections.singletonMap("key", 1);
Map<Object, Object> offsetData = Map.of("key", 1);
// Expect no exception to be thrown
OffsetUtils.validateFormat(offsetData);
}
@ -99,17 +99,17 @@ public class OffsetUtilsTest {
@Test
public void testProcessPartitionKeyListWithOneElement() {
assertInvalidPartitionKey(
serializePartitionKey(Collections.singletonList("")),
serializePartitionKey(List.of("")),
"Ignoring offset partition key with an unexpected number of elements");
}
@Test
public void testProcessPartitionKeyListWithElementsOfWrongType() {
assertInvalidPartitionKey(
serializePartitionKey(Arrays.asList(1, new HashMap<>())),
serializePartitionKey(List.of(1, new HashMap<>())),
"Ignoring offset partition key with an unexpected format for the first element in the partition key list");
assertInvalidPartitionKey(
serializePartitionKey(Arrays.asList("connector-name", new ArrayList<>())),
serializePartitionKey(List.of("connector-name", new ArrayList<>())),
"Ignoring offset partition key with an unexpected format for the second element in the partition key list");
}
@ -128,7 +128,7 @@ public class OffsetUtilsTest {
public void testProcessPartitionKeyValidList() {
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) {
Map<String, Set<Map<String, Object>>> connectorPartitions = new HashMap<>();
OffsetUtils.processPartitionKey(serializePartitionKey(Arrays.asList("connector-name", new HashMap<>())), new byte[0], CONVERTER, connectorPartitions);
OffsetUtils.processPartitionKey(serializePartitionKey(List.of("connector-name", new HashMap<>())), new byte[0], CONVERTER, connectorPartitions);
assertEquals(1, connectorPartitions.size());
assertEquals(0, logCaptureAppender.getMessages().size());
}
@ -139,7 +139,7 @@ public class OffsetUtilsTest {
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) {
Map<String, Set<Map<String, Object>>> connectorPartitions = new HashMap<>();
OffsetUtils.processPartitionKey(serializePartitionKey(Arrays.asList("connector-name", null)), new byte[0], CONVERTER, connectorPartitions);
assertEquals(Collections.emptyMap(), connectorPartitions);
assertEquals(Map.of(), connectorPartitions);
assertEquals(0, logCaptureAppender.getMessages().size());
}
}

View File

@ -27,7 +27,6 @@ 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.Map;
import java.util.Optional;
@ -125,7 +124,7 @@ public class ConnectUtilsTest {
"thanks to newly-introduced federal legislation",
false)
);
assertEquals(Collections.singletonMap("\u1984", "big brother"), props);
assertEquals(Map.of("\u1984", "big brother"), props);
props.clear();
props.put("\u1984", "BIG BROTHER");
@ -141,7 +140,7 @@ public class ConnectUtilsTest {
"thanks to newly-introduced federal legislation",
true)
);
assertEquals(Collections.singletonMap("\u1984", "big brother"), props);
assertEquals(Map.of("\u1984", "big brother"), props);
}
@Test

View File

@ -49,7 +49,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -152,7 +151,7 @@ public class KafkaBasedLogTest {
}
};
consumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name());
consumer.updatePartitions(TOPIC, Arrays.asList(TPINFO0, TPINFO1));
consumer.updatePartitions(TOPIC, List.of(TPINFO0, TPINFO1));
Map<TopicPartition, Long> beginningOffsets = new HashMap<>();
beginningOffsets.put(TP0, 0L);
beginningOffsets.put(TP1, 0L);

View File

@ -26,7 +26,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.time.Duration;
import java.util.Collections;
import java.util.Map;
import java.util.function.Function;
@ -43,7 +42,7 @@ import static org.mockito.Mockito.when;
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class SharedTopicAdminTest {
private static final Map<String, Object> EMPTY_CONFIG = Collections.emptyMap();
private static final Map<String, Object> EMPTY_CONFIG = Map.of();
@Mock private TopicAdmin mockTopicAdmin;
@Mock private Function<Map<String, Object>, TopicAdmin> factory;

View File

@ -23,7 +23,6 @@ import org.apache.kafka.connect.runtime.rest.entities.ConnectorOffsets;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -44,7 +43,7 @@ public class SinkUtilsTest {
connectorOffsets = SinkUtils.consumerGroupOffsetsToConnectorOffsets(consumerGroupOffsets);
assertEquals(1, connectorOffsets.offsets().size());
assertEquals(Collections.singletonMap(SinkUtils.KAFKA_OFFSET_KEY, 100L), connectorOffsets.offsets().get(0).offset());
assertEquals(Map.of(SinkUtils.KAFKA_OFFSET_KEY, 100L), connectorOffsets.offsets().get(0).offset());
Map<String, Object> expectedPartition = new HashMap<>();
expectedPartition.put(SinkUtils.KAFKA_TOPIC_KEY, "test-topic");

View File

@ -44,5 +44,4 @@ public class TableTest {
assertNull(table.get("foo", 6));
assertTrue(table.row("foo").isEmpty());
}
}

View File

@ -69,7 +69,6 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic;
@ -161,8 +160,8 @@ public class TopicAdminTest {
NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build();
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, "myTopic", Collections.singletonList(topicPartitionInfo), null);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, "myTopic", List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
assertFalse(admin.createTopic(newTopic));
assertTrue(admin.createTopics(newTopic).isEmpty());
@ -307,12 +306,12 @@ public class TopicAdminTest {
NewTopic newTopic = TopicAdmin.defineTopic(topicName).partitions(1).compacted().build();
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
Map<String, TopicDescription> desc = admin.describeTopics(newTopic.name());
assertFalse(desc.isEmpty());
TopicDescription topicDesc = new TopicDescription(topicName, false, Collections.singletonList(topicPartitionInfo));
TopicDescription topicDesc = new TopicDescription(topicName, false, List.of(topicPartitionInfo));
assertEquals(desc.get("myTopic"), topicDesc);
}
}
@ -381,14 +380,14 @@ public class TopicAdminTest {
public void describeTopicConfigShouldReturnTopicConfigWhenTopicExists() {
String topicName = "myTopic";
NewTopic newTopic = TopicAdmin.defineTopic(topicName)
.config(Collections.singletonMap("foo", "bar"))
.config(Map.of("foo", "bar"))
.partitions(1)
.compacted()
.build();
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
Map<String, Config> result = admin.describeTopicConfigs(newTopic.name());
assertFalse(result.isEmpty());
@ -438,11 +437,11 @@ public class TopicAdminTest {
@Test
public void verifyingTopicCleanupPolicyShouldReturnTrueWhenTopicHasCorrectPolicy() {
String topicName = "myTopic";
Map<String, String> topicConfigs = Collections.singletonMap("cleanup.policy", "compact");
Map<String, String> topicConfigs = Map.of("cleanup.policy", "compact");
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
boolean result = admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose");
assertTrue(result);
@ -452,11 +451,11 @@ public class TopicAdminTest {
@Test
public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeletePolicy() {
String topicName = "myTopic";
Map<String, String> topicConfigs = Collections.singletonMap("cleanup.policy", "delete");
Map<String, String> topicConfigs = Map.of("cleanup.policy", "delete");
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
@ -466,11 +465,11 @@ public class TopicAdminTest {
@Test
public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeleteAndCompactPolicy() {
String topicName = "myTopic";
Map<String, String> topicConfigs = Collections.singletonMap("cleanup.policy", "delete,compact");
Map<String, String> topicConfigs = Map.of("cleanup.policy", "delete,compact");
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
@ -480,11 +479,11 @@ public class TopicAdminTest {
@Test
public void verifyingGettingTopicCleanupPolicies() {
String topicName = "myTopic";
Map<String, String> topicConfigs = Collections.singletonMap("cleanup.policy", "compact");
Map<String, String> topicConfigs = Map.of("cleanup.policy", "compact");
Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient);
Set<String> policies = admin.topicCleanupPolicy("myTopic");
assertEquals(1, policies.size());
@ -501,7 +500,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldRethrowUnknownVersionException() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -519,7 +518,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldWrapNonRetriableExceptionsWithConnectException() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = 1000L;
Cluster cluster = createCluster(1, "myTopic", 1);
@ -548,7 +547,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldRetryWhenTopicNotFound() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = 1000L;
Cluster cluster = createCluster(1, "myTopic", 1);
@ -560,7 +559,7 @@ public class TopicAdminTest {
TopicAdmin admin = new TopicAdmin(env.adminClient());
Map<TopicPartition, Long> endoffsets = admin.retryEndOffsets(tps, Duration.ofMillis(100), 1);
assertEquals(Collections.singletonMap(tp1, offset), endoffsets);
assertEquals(Map.of(tp1, offset), endoffsets);
}
}
@ -568,7 +567,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithNonRetriableWhenAuthorizationFailureOccurs() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -585,7 +584,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithUnsupportedVersionWhenVersionUnsupportedErrorOccurs() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -601,7 +600,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithTimeoutExceptionWhenTimeoutErrorOccurs() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(
@ -619,7 +618,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithNonRetriableWhenUnknownErrorOccurs() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -638,7 +637,7 @@ public class TopicAdminTest {
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
TopicAdmin admin = new TopicAdmin(env.adminClient());
Map<TopicPartition, Long> offsets = admin.endOffsets(Collections.emptySet());
Map<TopicPartition, Long> offsets = admin.endOffsets(Set.of());
assertTrue(offsets.isEmpty());
}
}
@ -647,7 +646,7 @@ public class TopicAdminTest {
public void endOffsetsShouldReturnOffsetsForOnePartition() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
long offset = 1000L;
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -686,7 +685,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWhenAnyTopicPartitionHasError() {
String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1);
Set<TopicPartition> tps = Set.of(tp1);
Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
@ -718,8 +717,8 @@ public class TopicAdminTest {
"mockClusterId",
nodes.values(),
pInfos,
Collections.emptySet(),
Collections.emptySet(),
Set.of(),
Set.of(),
leader);
}
@ -737,9 +736,9 @@ public class TopicAdminTest {
.setPartitionIndex(pInfo.partition())
.setLeaderId(pInfo.leader().id())
.setLeaderEpoch(234)
.setReplicaNodes(Arrays.stream(pInfo.replicas()).map(Node::id).collect(Collectors.toList()))
.setIsrNodes(Arrays.stream(pInfo.inSyncReplicas()).map(Node::id).collect(Collectors.toList()))
.setOfflineReplicas(Arrays.stream(pInfo.offlineReplicas()).map(Node::id).collect(Collectors.toList()));
.setReplicaNodes(Arrays.stream(pInfo.replicas()).map(Node::id).toList())
.setIsrNodes(Arrays.stream(pInfo.inSyncReplicas()).map(Node::id).toList())
.setOfflineReplicas(Arrays.stream(pInfo.offlineReplicas()).map(Node::id).toList());
pms.add(pm);
}
MetadataResponseTopic tm = new MetadataResponseTopic()
@ -787,7 +786,7 @@ public class TopicAdminTest {
}
private ListOffsetsResponse listOffsetsResult(TopicPartition tp1, Long offset1) {
return listOffsetsResult(null, Collections.singletonMap(tp1, offset1));
return listOffsetsResult(null, Map.of(tp1, offset1));
}
private ListOffsetsResponse listOffsetsResult(TopicPartition tp1, Long offset1, TopicPartition tp2, Long offset2) {
@ -889,7 +888,7 @@ public class TopicAdminTest {
protected TopicDescription topicDescription(MockAdminClient admin, String topicName)
throws ExecutionException, InterruptedException {
DescribeTopicsResult result = admin.describeTopics(Collections.singleton(topicName));
DescribeTopicsResult result = admin.describeTopics(Set.of(topicName));
Map<String, KafkaFuture<TopicDescription>> byName = result.topicNameValues();
return byName.get(topicName).get();
}
@ -958,8 +957,8 @@ public class TopicAdminTest {
.map(e -> new DescribeConfigsResponseData.DescribeConfigsResourceResult()
.setName(e.getKey())
.setValue(e.getValue()))
.collect(Collectors.toList())))
.collect(Collectors.toList());
.toList()))
.toList();
return new DescribeConfigsResponse(new DescribeConfigsResponseData().setThrottleTimeMs(1000).setResults(results));
}

View File

@ -33,7 +33,6 @@ import org.apache.kafka.connect.transforms.RegexRouter;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -154,7 +153,7 @@ public class TopicCreationTest {
assertFalse(topicCreation.isTopicCreationEnabled());
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertNull(topicCreation.defaultTopicGroup());
assertEquals(Collections.emptyMap(), topicCreation.topicGroups());
assertEquals(Map.of(), topicCreation.topicGroups());
assertNull(topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -169,7 +168,7 @@ public class TopicCreationTest {
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertNull(topicCreation.defaultTopicGroup());
assertEquals(0, topicCreation.topicGroups().size());
assertEquals(Collections.emptyMap(), topicCreation.topicGroups());
assertEquals(Map.of(), topicCreation.topicGroups());
assertNull(topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -187,14 +186,14 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(DEFAULT_PARTITIONS, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(1, groups.size());
assertEquals(Collections.singleton(DEFAULT_TOPIC_CREATION_GROUP), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -205,7 +204,7 @@ public class TopicCreationTest {
assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name());
assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(Collections.emptyMap(), topicCreation.topicGroups());
assertEquals(Map.of(), topicCreation.topicGroups());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -215,7 +214,7 @@ public class TopicCreationTest {
assertEquals(FOO_TOPIC, topicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, topicSpec.replicationFactor());
assertEquals(DEFAULT_PARTITIONS, topicSpec.numPartitions());
assertEquals(Collections.emptyMap(), topicSpec.configs());
assertEquals(Map.of(), topicSpec.configs());
}
@Test
@ -241,14 +240,14 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(replicas, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(topicProps, sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(1, groups.size());
assertEquals(Collections.singleton(DEFAULT_TOPIC_CREATION_GROUP), groups.keySet());
assertEquals(Set.of(DEFAULT_TOPIC_CREATION_GROUP), groups.keySet());
// verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -259,7 +258,7 @@ public class TopicCreationTest {
assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name());
assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(Collections.emptyMap(), topicCreation.topicGroups());
assertEquals(Map.of(), topicCreation.topicGroups());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -292,9 +291,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -318,7 +317,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(1, topicCreation.topicGroups().size());
assertEquals(Collections.singleton(FOO_GROUP), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP), topicCreation.topicGroups().keySet());
assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -328,7 +327,7 @@ public class TopicCreationTest {
assertEquals(BAR_TOPIC, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs());
assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name());
@ -357,9 +356,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -384,7 +383,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC));
assertEquals(1, topicCreation.topicGroups().size());
assertEquals(Collections.singleton(FOO_GROUP), topicCreation.topicGroups().keySet());
assertEquals(Set.of(FOO_GROUP), topicCreation.topicGroups().keySet());
assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC));
assertEquals(fooGroup, topicCreation.findFirstGroup(BAR_TOPIC));
topicCreation.addTopic(FOO_TOPIC);
@ -434,9 +433,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -479,7 +478,7 @@ public class TopicCreationTest {
assertEquals(otherTopic, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs());
assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name());
@ -566,9 +565,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -611,7 +610,7 @@ public class TopicCreationTest {
assertEquals(otherTopic, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs());
assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name());

View File

@ -47,7 +47,6 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@ -282,7 +281,7 @@ abstract class EmbeddedConnect {
throw new ConnectException("Failed to serialize connector creation request: " + createConnectorRequest);
}
Response response = requestPost(url, requestBody, Collections.emptyMap());
Response response = requestPost(url, requestBody, Map.of());
if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) {
return responseToString(response);
} else {
@ -449,7 +448,7 @@ abstract class EmbeddedConnect {
*/
public void restartConnector(String connName) {
String url = endpointForResource(String.format("connectors/%s/restart", connName));
Response response = requestPost(url, "", Collections.emptyMap());
Response response = requestPost(url, "", Map.of());
if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) {
throw new ConnectRestException(response.getStatus(),
"Could not execute POST request. Error response: " + responseToString(response));
@ -466,7 +465,7 @@ abstract class EmbeddedConnect {
*/
public void restartTask(String connName, int taskNum) {
String url = endpointForResource(String.format("connectors/%s/tasks/%d/restart", connName, taskNum));
Response response = requestPost(url, "", Collections.emptyMap());
Response response = requestPost(url, "", Map.of());
if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) {
throw new ConnectRestException(response.getStatus(),
"Could not execute POST request. Error response: " + responseToString(response));
@ -492,7 +491,7 @@ abstract class EmbeddedConnect {
} else {
restartEndpoint = endpointForResource(restartPath);
}
Response response = requestPost(restartEndpoint, "", Collections.emptyMap());
Response response = requestPost(restartEndpoint, "", Map.of());
try {
if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) {
//only the 202 status returns a body
@ -577,7 +576,7 @@ abstract class EmbeddedConnect {
.readerFor(new TypeReference<Map<String, Map<String, List<String>>>>() { })
.readValue(responseToString(response));
return new ActiveTopicsInfo(connectorName,
activeTopics.get(connectorName).getOrDefault("topics", Collections.emptyList()));
activeTopics.get(connectorName).getOrDefault("topics", List.of()));
}
} catch (IOException e) {
log.error("Could not read connector state from response: {}",
@ -688,7 +687,7 @@ abstract class EmbeddedConnect {
public String alterSourceConnectorOffset(String connectorName, Map<String, ?> partition, Map<String, ?> offset) {
return alterConnectorOffsets(
connectorName,
new ConnectorOffsets(Collections.singletonList(new ConnectorOffset(partition, offset)))
new ConnectorOffsets(List.of(new ConnectorOffset(partition, offset)))
);
}
@ -705,7 +704,7 @@ abstract class EmbeddedConnect {
public String alterSinkConnectorOffset(String connectorName, TopicPartition topicPartition, Long offset) {
return alterConnectorOffsets(
connectorName,
SinkUtils.consumerGroupOffsetsToConnectorOffsets(Collections.singletonMap(topicPartition, new OffsetAndMetadata(offset)))
SinkUtils.consumerGroupOffsetsToConnectorOffsets(Map.of(topicPartition, new OffsetAndMetadata(offset)))
);
}
@ -929,7 +928,7 @@ abstract class EmbeddedConnect {
* @throws ConnectException if execution of the GET request fails
*/
public Response requestGet(String url) {
return requestHttpMethod(url, null, Collections.emptyMap(), "GET");
return requestHttpMethod(url, null, Map.of(), "GET");
}
/**
@ -941,7 +940,7 @@ abstract class EmbeddedConnect {
* @throws ConnectException if execution of the PUT request fails
*/
public Response requestPut(String url, String body) {
return requestHttpMethod(url, body, Collections.emptyMap(), "PUT");
return requestHttpMethod(url, body, Map.of(), "PUT");
}
/**
@ -966,7 +965,7 @@ abstract class EmbeddedConnect {
* @throws ConnectException if execution of the PATCH request fails
*/
public Response requestPatch(String url, String body) {
return requestHttpMethod(url, body, Collections.emptyMap(), "PATCH");
return requestHttpMethod(url, body, Map.of(), "PATCH");
}
/**
@ -977,7 +976,7 @@ abstract class EmbeddedConnect {
* @throws ConnectException if execution of the DELETE request fails
*/
public Response requestDelete(String url) {
return requestHttpMethod(url, null, Collections.emptyMap(), "DELETE");
return requestHttpMethod(url, null, Map.of(), "DELETE");
}
/**

View File

@ -31,7 +31,6 @@ import java.io.OutputStream;
import java.io.UncheckedIOException;
import java.nio.file.Files;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
@ -109,8 +108,8 @@ public class EmbeddedConnectStandalone extends EmbeddedConnect {
@Override
protected Set<WorkerHandle> workers() {
return connectWorker != null
? Collections.singleton(connectWorker)
: Collections.emptySet();
? Set.of(connectWorker)
: Set.of();
}
public Response healthCheck() {

View File

@ -63,7 +63,6 @@ import java.time.Duration;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -111,7 +110,7 @@ public class EmbeddedKafkaCluster {
private KafkaProducer<byte[], byte[]> producer;
public EmbeddedKafkaCluster(final int numBrokers, final Properties brokerConfig) {
this(numBrokers, brokerConfig, Collections.emptyMap());
this(numBrokers, brokerConfig, Map.of());
}
public EmbeddedKafkaCluster(final int numBrokers,
@ -180,7 +179,7 @@ public class EmbeddedKafkaCluster {
*/
public void verifyClusterReadiness() {
String consumerGroupId = UUID.randomUUID().toString();
Map<String, Object> consumerConfig = Collections.singletonMap(GROUP_ID_CONFIG, consumerGroupId);
Map<String, Object> consumerConfig = Map.of(GROUP_ID_CONFIG, consumerGroupId);
String topic = "consumer-warmup-" + consumerGroupId;
try {
@ -204,8 +203,8 @@ public class EmbeddedKafkaCluster {
}
try (Admin admin = createAdminClient()) {
admin.deleteConsumerGroups(Collections.singleton(consumerGroupId)).all().get(30, TimeUnit.SECONDS);
admin.deleteTopics(Collections.singleton(topic)).all().get(30, TimeUnit.SECONDS);
admin.deleteConsumerGroups(Set.of(consumerGroupId)).all().get(30, TimeUnit.SECONDS);
admin.deleteTopics(Set.of(topic)).all().get(30, TimeUnit.SECONDS);
} catch (InterruptedException | ExecutionException | TimeoutException e) {
throw new AssertionError("Failed to clean up cluster health check resource(s)", e);
}
@ -354,7 +353,7 @@ public class EmbeddedKafkaCluster {
* @param topic The name of the topic.
*/
public void createTopic(String topic, int partitions) {
createTopic(topic, partitions, 1, Collections.emptyMap());
createTopic(topic, partitions, 1, Map.of());
}
/**
@ -363,7 +362,7 @@ public class EmbeddedKafkaCluster {
* @param topic The name of the topic.
*/
public void createTopic(String topic, int partitions, int replication, Map<String, String> topicConfig) {
createTopic(topic, partitions, replication, topicConfig, Collections.emptyMap());
createTopic(topic, partitions, replication, topicConfig, Map.of());
}
/**
@ -387,7 +386,7 @@ public class EmbeddedKafkaCluster {
newTopic.configs(topicConfig);
try (final Admin adminClient = createAdminClient(adminClientConfig)) {
adminClient.createTopics(Collections.singletonList(newTopic)).all().get();
adminClient.createTopics(List.of(newTopic)).all().get();
} catch (final InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
@ -400,7 +399,7 @@ public class EmbeddedKafkaCluster {
*/
public void deleteTopic(String topic) {
try (final Admin adminClient = createAdminClient()) {
adminClient.deleteTopics(Collections.singleton(topic)).all().get();
adminClient.deleteTopics(Set.of(topic)).all().get();
} catch (final InterruptedException | ExecutionException e) {
throw new RuntimeException(e);
}
@ -436,7 +435,7 @@ public class EmbeddedKafkaCluster {
}
public Admin createAdminClient() {
return createAdminClient(Collections.emptyMap());
return createAdminClient(Map.of());
}
/**
@ -448,7 +447,7 @@ public class EmbeddedKafkaCluster {
* @return a {@link ConsumerRecords} collection containing at least n records.
*/
public ConsumerRecords<byte[], byte[]> consume(int n, long maxDuration, String... topics) {
return consume(n, maxDuration, Collections.emptyMap(), topics);
return consume(n, maxDuration, Map.of(), topics);
}
/**
@ -524,10 +523,10 @@ public class EmbeddedKafkaCluster {
long remainingTimeMs;
Set<TopicPartition> topicPartitions;
Map<TopicPartition, Long> endOffsets;
try (Admin admin = createAdminClient(adminProps != null ? adminProps : Collections.emptyMap())) {
try (Admin admin = createAdminClient(adminProps != null ? adminProps : Map.of())) {
remainingTimeMs = endTimeMs - System.currentTimeMillis();
topicPartitions = listPartitions(remainingTimeMs, admin, Arrays.asList(topics));
topicPartitions = listPartitions(remainingTimeMs, admin, List.of(topics));
remainingTimeMs = endTimeMs - System.currentTimeMillis();
endOffsets = readEndOffsets(remainingTimeMs, admin, topicPartitions);
@ -539,7 +538,7 @@ public class EmbeddedKafkaCluster {
tp -> new ArrayList<>()
));
Map<TopicPartition, OffsetAndMetadata> nextOffsets = new HashMap<>();
try (Consumer<byte[], byte[]> consumer = createConsumer(consumerProps != null ? consumerProps : Collections.emptyMap())) {
try (Consumer<byte[], byte[]> consumer = createConsumer(consumerProps != null ? consumerProps : Map.of())) {
consumer.assign(topicPartitions);
while (!endOffsets.isEmpty()) {
@ -555,7 +554,7 @@ public class EmbeddedKafkaCluster {
} else {
remainingTimeMs = endTimeMs - System.currentTimeMillis();
if (remainingTimeMs <= 0) {
throw new AssertionError("failed to read to end of topic(s) " + Arrays.asList(topics) + " within " + maxDurationMs + "ms");
throw new AssertionError("failed to read to end of topic(s) " + List.of(topics) + " within " + maxDurationMs + "ms");
}
// We haven't reached the end offset yet; need to keep polling
ConsumerRecords<byte[], byte[]> recordBatch = consumer.poll(Duration.ofMillis(remainingTimeMs));
@ -573,7 +572,7 @@ public class EmbeddedKafkaCluster {
public long endOffset(TopicPartition topicPartition) throws TimeoutException, InterruptedException, ExecutionException {
try (Admin admin = createAdminClient()) {
Map<TopicPartition, OffsetSpec> offsets = Collections.singletonMap(
Map<TopicPartition, OffsetSpec> offsets = Map.of(
topicPartition, OffsetSpec.latest()
);
return admin.listOffsets(offsets)
@ -662,9 +661,9 @@ public class EmbeddedKafkaCluster {
public KafkaConsumer<byte[], byte[]> createConsumerAndSubscribeTo(Map<String, Object> consumerProps, ConsumerRebalanceListener rebalanceListener, String... topics) {
KafkaConsumer<byte[], byte[]> consumer = createConsumer(consumerProps);
if (rebalanceListener != null) {
consumer.subscribe(Arrays.asList(topics), rebalanceListener);
consumer.subscribe(List.of(topics), rebalanceListener);
} else {
consumer.subscribe(Arrays.asList(topics));
consumer.subscribe(List.of(topics));
}
return consumer;
}

View File

@ -50,8 +50,8 @@ public class ReadVersionFromResource implements Converter, Versioned {
try (BufferedReader reader = new BufferedReader(new InputStreamReader(stream))) {
return reader.lines()
.filter(s -> !s.isEmpty() && !s.startsWith("#"))
.collect(Collectors.toList())
.get(0);
.findFirst()
.get();
}
}

View File

@ -50,8 +50,8 @@ public class ReadVersionFromResource implements Converter, Versioned {
try (BufferedReader reader = new BufferedReader(new InputStreamReader(stream))) {
return reader.lines()
.filter(s -> !s.isEmpty() && !s.startsWith("#"))
.collect(Collectors.toList())
.get(0);
.findFirst()
.get();
}
}

View File

@ -3145,7 +3145,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
return 0L;
}
long timeUtilVoterChangeExpires = state.maybeExpirePendingOperation(currentTimeMs);
long timeUntilVoterChangeExpires = state.maybeExpirePendingOperation(currentTimeMs);
long timeUntilFlush = maybeAppendBatches(
state,
@ -3163,7 +3163,7 @@ public final class KafkaRaftClient<T> implements RaftClient<T> {
timeUntilNextBeginQuorumSend,
Math.min(
timeUntilCheckQuorumExpires,
timeUtilVoterChangeExpires
timeUntilVoterChangeExpires
)
)
);

View File

@ -2038,9 +2038,6 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
leaderCopyRLMTasks.values().forEach(RLMTaskWithFuture::cancel);
leaderExpirationRLMTasks.values().forEach(RLMTaskWithFuture::cancel);
followerRLMTasks.values().forEach(RLMTaskWithFuture::cancel);
Utils.closeQuietly(remoteStorageManagerPlugin, "remoteStorageManagerPlugin");
Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin");
Utils.closeQuietly(indexCache, "RemoteIndexCache");
rlmCopyThreadPool.close();
rlmExpirationThreadPool.close();
@ -2050,10 +2047,13 @@ public class RemoteLogManager implements Closeable, AsyncOffsetReader {
} finally {
removeMetrics();
}
leaderCopyRLMTasks.clear();
leaderExpirationRLMTasks.clear();
followerRLMTasks.clear();
Utils.closeQuietly(indexCache, "RemoteIndexCache");
Utils.closeQuietly(remoteLogMetadataManagerPlugin, "remoteLogMetadataManagerPlugin");
Utils.closeQuietly(remoteStorageManagerPlugin, "remoteStorageManagerPlugin");
closed = true;
}
}

View File

@ -1769,9 +1769,9 @@ public class RemoteLogManagerTest {
void testIdempotentClose() throws IOException {
remoteLogManager.close();
remoteLogManager.close();
InOrder inorder = inOrder(remoteStorageManager, remoteLogMetadataManager);
inorder.verify(remoteStorageManager, times(1)).close();
InOrder inorder = inOrder(remoteLogMetadataManager, remoteStorageManager);
inorder.verify(remoteLogMetadataManager, times(1)).close();
inorder.verify(remoteStorageManager, times(1)).close();
}
@Test

View File

@ -73,14 +73,13 @@ public class AclCommand {
public static void main(String[] args) {
AclCommandOptions opts = new AclCommandOptions(args);
AdminClientService aclCommandService = new AdminClientService(opts);
try (Admin admin = Admin.create(adminConfigs(opts))) {
if (opts.options.has(opts.addOpt)) {
aclCommandService.addAcls(admin);
addAcls(admin, opts);
} else if (opts.options.has(opts.removeOpt)) {
aclCommandService.removeAcls(admin);
removeAcls(admin, opts);
} else if (opts.options.has(opts.listOpt)) {
aclCommandService.listAcls(admin);
listAcls(admin, opts);
}
} catch (Throwable e) {
System.out.println("Error while executing ACL command: " + e.getMessage());
@ -102,15 +101,7 @@ public class AclCommand {
return props;
}
private static class AdminClientService {
private final AclCommandOptions opts;
AdminClientService(AclCommandOptions opts) {
this.opts = opts;
}
void addAcls(Admin admin) throws ExecutionException, InterruptedException {
private static void addAcls(Admin admin, AclCommandOptions opts) throws ExecutionException, InterruptedException {
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcl = getResourceToAcls(opts);
for (Map.Entry<ResourcePattern, Set<AccessControlEntry>> entry : resourceToAcl.entrySet()) {
ResourcePattern resource = entry.getKey();
@ -121,7 +112,7 @@ public class AclCommand {
}
}
void removeAcls(Admin admin) throws ExecutionException, InterruptedException {
private static void removeAcls(Admin admin, AclCommandOptions opts) throws ExecutionException, InterruptedException {
Map<ResourcePatternFilter, Set<AccessControlEntry>> filterToAcl = getResourceFilterToAcls(opts);
for (Map.Entry<ResourcePatternFilter, Set<AccessControlEntry>> entry : filterToAcl.entrySet()) {
ResourcePatternFilter filter = entry.getKey();
@ -141,7 +132,7 @@ public class AclCommand {
}
}
private void listAcls(Admin admin) throws ExecutionException, InterruptedException {
private static void listAcls(Admin admin, AclCommandOptions opts) throws ExecutionException, InterruptedException {
Set<ResourcePatternFilter> filters = getResourceFilter(opts, false);
Set<KafkaPrincipal> listPrincipals = getPrincipals(opts, opts.listPrincipalsOpt);
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls = getAcls(admin, filters);
@ -182,7 +173,7 @@ public class AclCommand {
}
}
private Map<ResourcePattern, Set<AccessControlEntry>> getAcls(Admin adminClient, Set<ResourcePatternFilter> filters) throws ExecutionException, InterruptedException {
private static Map<ResourcePattern, Set<AccessControlEntry>> getAcls(Admin adminClient, Set<ResourcePatternFilter> filters) throws ExecutionException, InterruptedException {
Collection<AclBinding> aclBindings;
if (filters.isEmpty()) {
aclBindings = adminClient.describeAcls(AclBindingFilter.ANY).values().get();
@ -202,7 +193,6 @@ public class AclCommand {
}
return resourceToAcls;
}
}
private static Map<ResourcePattern, Set<AccessControlEntry>> getResourceToAcls(AclCommandOptions opts) {
PatternType patternType = opts.options.valueOf(opts.resourcePatternType);

View File

@ -196,22 +196,8 @@ public class ConnectPluginPath {
LIST, SYNC_MANIFESTS
}
private static class Config {
private final Command command;
private final Set<Path> locations;
private final boolean dryRun;
private final boolean keepNotFound;
private final PrintStream out;
private final PrintStream err;
private Config(Command command, Set<Path> locations, boolean dryRun, boolean keepNotFound, PrintStream out, PrintStream err) {
this.command = command;
this.locations = locations;
this.dryRun = dryRun;
this.keepNotFound = keepNotFound;
this.out = out;
this.err = err;
}
private record Config(Command command, Set<Path> locations, boolean dryRun, boolean keepNotFound, PrintStream out,
PrintStream err) {
@Override
public String toString() {
@ -262,16 +248,9 @@ public class ConnectPluginPath {
* <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information
* that pertains to this specific plugin.
*/
private static class Row {
private final ManifestWorkspace.SourceWorkspace<?> workspace;
private final String className;
private final PluginType type;
private final String version;
private final List<String> aliases;
private final boolean loadable;
private final boolean hasManifest;
public Row(ManifestWorkspace.SourceWorkspace<?> workspace, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) {
private record Row(ManifestWorkspace.SourceWorkspace<?> workspace, String className, PluginType type,
String version, List<String> aliases, boolean loadable, boolean hasManifest) {
private Row(ManifestWorkspace.SourceWorkspace<?> workspace, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) {
this.workspace = Objects.requireNonNull(workspace, "workspace must be non-null");
this.className = Objects.requireNonNull(className, "className must be non-null");
this.version = Objects.requireNonNull(version, "version must be non-null");
@ -281,10 +260,6 @@ public class ConnectPluginPath {
this.hasManifest = hasManifest;
}
private boolean loadable() {
return loadable;
}
private boolean compatible() {
return loadable && hasManifest;
}

View File

@ -65,7 +65,6 @@ import java.util.stream.Stream;
import static java.lang.String.format;
import static java.lang.String.valueOf;
import static java.util.Arrays.asList;
/**
* A tool for describing quorum status
@ -206,7 +205,7 @@ public class MetadataQuorumCommand {
rows.addAll(quorumInfoToRows(leader, quorumInfo.observers().stream(), "Observer", humanReadable));
ToolsUtils.prettyPrintTable(
asList("NodeId", "DirectoryId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
List.of("NodeId", "DirectoryId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
rows,
System.out
);

View File

@ -292,14 +292,7 @@ public class OAuthCompatibilityTool {
}
private static class ConfigHandler {
private final Namespace namespace;
private ConfigHandler(Namespace namespace) {
this.namespace = namespace;
}
private record ConfigHandler(Namespace namespace) {
private Map<String, ?> getConfigs() {
Map<String, Object> m = new HashMap<>();

View File

@ -500,16 +500,7 @@ public class OffsetsUtils {
public interface LogOffsetResult { }
public static class LogOffset implements LogOffsetResult {
final long value;
public LogOffset(long value) {
this.value = value;
}
public long value() {
return value;
}
public record LogOffset(long value) implements LogOffsetResult {
}
public static class Unknown implements LogOffsetResult { }

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