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 org.apache.kafka.common.TopicPartition;
import java.time.Duration;
import java.util.Collection; 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. * 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. * 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()}). * or is unsubscribing ({@link KafkaConsumer#unsubscribe()}).
* It is recommended that offsets should be committed in this callback to either Kafka or a * It is recommended that offsets should be committed in this callback to either Kafka or a
* custom offset store to prevent duplicate data. * custom offset store to prevent duplicate data.

View File

@ -24,7 +24,6 @@ import java.util.Iterator;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.function.BiConsumer; 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 // the request is now relevant. Partitions will be processed in the order
// they appear in the request. // 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. // able to handle Kafka log message format version 2.
// //
// Version 5 adds LogStartOffset to indicate the earliest available offset of // 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 org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -32,7 +31,7 @@ public class NoneConnectorClientConfigOverridePolicyTest extends BaseConnectorCl
@Test @Test
public void testNoOverrides() { public void testNoOverrides() {
testValidOverride(Collections.emptyMap()); testValidOverride(Map.of());
} }
@Test @Test

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -30,7 +30,6 @@ import org.junit.jupiter.api.TestInfo;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map; import java.util.Map;
@ -131,7 +130,7 @@ public class ConnectorRestartApiIntegrationTest {
// Call the Restart API // Call the Restart API
String restartEndpoint = connect.endpointForResource( String restartEndpoint = connect.endpointForResource(
String.format("connectors/%s/restart", connectorName)); 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()); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus());
} }
@ -152,7 +151,7 @@ public class ConnectorRestartApiIntegrationTest {
// Call the Restart API // Call the Restart API
String restartEndpoint = connect.endpointForResource( String restartEndpoint = connect.endpointForResource(
String.format("connectors/%s/restart?onlyFailed=" + onlyFailed + "&includeTasks=" + includeTasks, connectorName)); 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()); assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus());
} }
@ -213,7 +212,7 @@ public class ConnectorRestartApiIntegrationTest {
@Test @Test
public void testOneFailedTasksRestartOnlyOneTasks() throws Exception { 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); 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.nio.charset.StandardCharsets;
import java.time.Duration; import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -43,7 +41,6 @@ import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Properties; import java.util.Properties;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport; import java.util.stream.StreamSupport;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG; 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(FOO_TOPIC, NUM_TOPIC_PARTITIONS);
connect.kafka().createTopic(BAR_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); "Active topic set is not empty for connector: " + FOO_CONNECTOR);
// start a source connector // start a source connector
@ -124,8 +121,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time."); "Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.singletonList(FOO_TOPIC), connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
// start another source connector // start another source connector
connect.configureConnector(BAR_CONNECTOR, defaultSourceConnectorProps(BAR_TOPIC)); connect.configureConnector(BAR_CONNECTOR, defaultSourceConnectorProps(BAR_TOPIC));
@ -133,8 +130,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(BAR_CONNECTOR, NUM_TASKS, connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(BAR_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time."); "Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, Collections.singletonList(BAR_TOPIC), connect.assertions().assertConnectorActiveTopics(BAR_CONNECTOR, List.of(BAR_TOPIC),
"Active topic set is not: " + Collections.singletonList(BAR_TOPIC) + " for connector: " + BAR_CONNECTOR); "Active topic set is not: " + List.of(BAR_TOPIC) + " for connector: " + BAR_CONNECTOR);
// start a sink connector // start a sink connector
connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC, BAR_TOPIC)); connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC, BAR_TOPIC));
@ -142,8 +139,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS, connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time."); "Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Arrays.asList(FOO_TOPIC, BAR_TOPIC), connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC, BAR_TOPIC),
"Active topic set is not: " + Arrays.asList(FOO_TOPIC, BAR_TOPIC) + " for connector: " + SINK_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC, BAR_TOPIC) + " for connector: " + SINK_CONNECTOR);
// deleting a connector resets its active topics // deleting a connector resets its active topics
connect.deleteConnector(BAR_CONNECTOR); connect.deleteConnector(BAR_CONNECTOR);
@ -151,7 +148,7 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorDoesNotExist(BAR_CONNECTOR, connect.assertions().assertConnectorDoesNotExist(BAR_CONNECTOR,
"Connector wasn't deleted in time."); "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); "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 // 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 // reset active topics for the sink connector after one of the topics has become idle
connect.resetConnectorTopics(SINK_CONNECTOR); connect.resetConnectorTopics(SINK_CONNECTOR);
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC), connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
} }
@Test @Test
@ -177,7 +174,7 @@ public class ConnectorTopicsIntegrationTest {
connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS); connect.kafka().createTopic(FOO_TOPIC, NUM_TOPIC_PARTITIONS);
connect.kafka().createTopic(BAR_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); "Active topic set is not empty for connector: " + FOO_CONNECTOR);
// start a source connector // start a source connector
@ -186,8 +183,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time."); "Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, Collections.singletonList(FOO_TOPIC), connect.assertions().assertConnectorActiveTopics(FOO_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + FOO_CONNECTOR);
// start a sink connector // start a sink connector
connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC)); connect.configureConnector(SINK_CONNECTOR, defaultSinkConnectorProps(FOO_TOPIC));
@ -195,8 +192,8 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS, connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(SINK_CONNECTOR, NUM_TASKS,
"Connector tasks did not start in time."); "Connector tasks did not start in time.");
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC), connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
// deleting a connector resets its active topics // deleting a connector resets its active topics
connect.deleteConnector(FOO_CONNECTOR); connect.deleteConnector(FOO_CONNECTOR);
@ -204,7 +201,7 @@ public class ConnectorTopicsIntegrationTest {
connect.assertions().assertConnectorDoesNotExist(FOO_CONNECTOR, connect.assertions().assertConnectorDoesNotExist(FOO_CONNECTOR,
"Connector wasn't deleted in time."); "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); "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 // 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)); Exception e = assertThrows(ConnectRestException.class, () -> connect.resetConnectorTopics(SINK_CONNECTOR));
assertTrue(e.getMessage().contains("Topic tracking reset is disabled.")); assertTrue(e.getMessage().contains("Topic tracking reset is disabled."));
connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, Collections.singletonList(FOO_TOPIC), connect.assertions().assertConnectorActiveTopics(SINK_CONNECTOR, List.of(FOO_TOPIC),
"Active topic set is not: " + Collections.singletonList(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR); "Active topic set is not: " + List.of(FOO_TOPIC) + " for connector: " + SINK_CONNECTOR);
} }
@Test @Test
@ -252,7 +249,7 @@ public class ConnectorTopicsIntegrationTest {
public void assertNoTopicStatusInStatusTopic() { public void assertNoTopicStatusInStatusTopic() {
String statusTopic = workerProps.get(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG); String statusTopic = workerProps.get(DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG);
Consumer<byte[], byte[]> verifiableConsumer = connect.kafka().createConsumer( 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); List<PartitionInfo> partitionInfos = verifiableConsumer.partitionsFor(statusTopic);
if (partitionInfos.isEmpty()) { if (partitionInfos.isEmpty()) {
@ -260,7 +257,7 @@ public class ConnectorTopicsIntegrationTest {
} }
List<TopicPartition> partitions = partitionInfos.stream() List<TopicPartition> partitions = partitionInfos.stream()
.map(info -> new TopicPartition(info.topic(), info.partition())) .map(info -> new TopicPartition(info.topic(), info.partition()))
.collect(Collectors.toList()); .toList();
verifiableConsumer.assign(partitions); verifiableConsumer.assign(partitions);
// Based on the implementation of {@link org.apache.kafka.connect.util.KafkaBasedLog#readToLogEnd} // 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.io.Closeable;
import java.time.Duration; import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; 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 // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll( ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic topic
); );
@ -366,7 +365,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll( ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic topic
); );
@ -427,7 +426,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka().consumeAll( ConsumerRecords<byte[], byte[]> sourceRecords = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic topic
); );
@ -538,7 +537,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll( ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic topic
); );
@ -601,7 +600,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll( ConsumerRecords<byte[], byte[]> records = connect.kafka().consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic topic
); );
@ -664,7 +663,7 @@ public class ExactlyOnceSourceIntegrationTest {
String topic = "test-topic"; String topic = "test-topic";
try (Admin admin = connect.kafka().createAdminClient()) { 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<>(); 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 // Grant the connector's admin permissions to access the topics for its records and offsets
// Intentionally leave out permissions required for fencing // Intentionally leave out permissions required for fencing
try (Admin admin = connect.kafka().createAdminClient()) { try (Admin admin = connect.kafka().createAdminClient()) {
admin.createAcls(Arrays.asList( admin.createAcls(List.of(
new AclBinding( new AclBinding(
new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL), new ResourcePattern(ResourceType.TOPIC, topic, PatternType.LITERAL),
new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW) 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 // Now grant the necessary permissions for fencing to the connector's admin
try (Admin admin = connect.kafka().createAdminClient()) { try (Admin admin = connect.kafka().createAdminClient()) {
admin.createAcls(Arrays.asList( admin.createAcls(List.of(
new AclBinding( new AclBinding(
new ResourcePattern(ResourceType.TRANSACTIONAL_ID, Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, 0), PatternType.LITERAL), new ResourcePattern(ResourceType.TRANSACTIONAL_ID, Worker.taskTransactionalId(CLUSTER_GROUP_ID, CONNECTOR_NAME, 0), PatternType.LITERAL),
new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW) new AccessControlEntry("User:connector", "*", AclOperation.ALL, AclPermissionType.ALLOW)
@ -864,7 +863,7 @@ public class ExactlyOnceSourceIntegrationTest {
.consume( .consume(
MINIMUM_MESSAGES, MINIMUM_MESSAGES,
TimeUnit.MINUTES.toMillis(1), TimeUnit.MINUTES.toMillis(1),
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
"test-topic") "test-topic")
.count(); .count();
assertTrue(recordNum >= MINIMUM_MESSAGES, assertTrue(recordNum >= MINIMUM_MESSAGES,
@ -874,7 +873,7 @@ public class ExactlyOnceSourceIntegrationTest {
ConsumerRecords<byte[], byte[]> offsetRecords = connectorTargetedCluster ConsumerRecords<byte[], byte[]> offsetRecords = connectorTargetedCluster
.consumeAll( .consumeAll(
TimeUnit.MINUTES.toMillis(1), TimeUnit.MINUTES.toMillis(1),
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
offsetsTopic offsetsTopic
); );
@ -930,7 +929,7 @@ public class ExactlyOnceSourceIntegrationTest {
// consume all records from the source topic or fail, to ensure that they were correctly produced // consume all records from the source topic or fail, to ensure that they were correctly produced
ConsumerRecords<byte[], byte[]> sourceRecords = connectorTargetedCluster.consumeAll( ConsumerRecords<byte[], byte[]> sourceRecords = connectorTargetedCluster.consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
topic 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 // also have to check which offsets have actually been committed, since we no longer have exactly-once semantics
offsetRecords = connectorTargetedCluster.consumeAll( offsetRecords = connectorTargetedCluster.consumeAll(
CONSUME_RECORDS_TIMEOUT_MS, CONSUME_RECORDS_TIMEOUT_MS,
Collections.singletonMap(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"), Map.of(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"),
null, null,
offsetsTopic offsetsTopic
); );
@ -999,13 +998,13 @@ public class ExactlyOnceSourceIntegrationTest {
private List<Long> parseAndAssertOffsetsForSingleTask(ConsumerRecords<byte[], byte[]> offsetRecords) { private List<Long> parseAndAssertOffsetsForSingleTask(ConsumerRecords<byte[], byte[]> offsetRecords) {
Map<Integer, List<Long>> parsedOffsets = parseOffsetForTasks(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); return parsedOffsets.get(0);
} }
private List<Long> parseAndAssertValuesForSingleTask(ConsumerRecords<byte[], byte[]> sourceRecords) { private List<Long> parseAndAssertValuesForSingleTask(ConsumerRecords<byte[], byte[]> sourceRecords) {
Map<Integer, List<Long>> parsedValues = parseValuesForTasks(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); return parsedValues.get(0);
} }
@ -1024,7 +1023,7 @@ public class ExactlyOnceSourceIntegrationTest {
parsedValues.replaceAll((task, values) -> { parsedValues.replaceAll((task, values) -> {
Long committedValue = lastCommittedValues.get(task); Long committedValue = lastCommittedValues.get(task);
assertNotNull(committedValue, "No committed offset found for task " + 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); assertSeqnos(parsedValues, numTasks);
} }
@ -1102,7 +1101,7 @@ public class ExactlyOnceSourceIntegrationTest {
JsonConverter offsetsConverter = new JsonConverter(); 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 // 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. // 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<>(); Map<Integer, List<Long>> result = new HashMap<>();
for (ConsumerRecord<byte[], byte[]> offsetRecord : offsetRecords) { for (ConsumerRecord<byte[], byte[]> offsetRecord : offsetRecords) {
@ -1284,7 +1283,7 @@ public class ExactlyOnceSourceIntegrationTest {
// Request a read to the end of the offsets topic // Request a read to the end of the offsets topic
context.offsetStorageReader().offset(Collections.singletonMap("", null)); context.offsetStorageReader().offset(Collections.singletonMap("", null));
// Produce a record to the offsets topic // 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 @Override

View File

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

View File

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

View File

@ -30,8 +30,8 @@ import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -350,8 +350,7 @@ public class RebalanceSourceConnectorsIntegrationTest {
private static String formatAssignment(Map<String, Collection<String>> assignment) { private static String formatAssignment(Map<String, Collection<String>> assignment) {
StringBuilder result = new StringBuilder(); StringBuilder result = new StringBuilder();
for (String worker : assignment.keySet().stream().sorted().toList()) { for (String worker : assignment.keySet().stream().sorted().toList()) {
result.append(String.format("\n%s=%s", worker, assignment.getOrDefault(worker, result.append(String.format("\n%s=%s", worker, assignment.getOrDefault(worker, List.of())));
Collections.emptyList())));
} }
return result.toString(); 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.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -112,7 +111,7 @@ public class RestExtensionIntegrationTest {
workerId, workerId,
null null
), ),
Collections.singletonMap( Map.of(
0, 0,
new TaskState(0, "RUNNING", workerId, null) new TaskState(0, "RUNNING", workerId, null)
), ),

View File

@ -63,7 +63,6 @@ import java.nio.charset.StandardCharsets;
import java.security.GeneralSecurityException; import java.security.GeneralSecurityException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -199,7 +198,7 @@ public class RestForwardingIntegrationTest {
.putConnectorConfig(any(), any(), isNull(), anyBoolean(), followerCallbackCaptor.capture()); .putConnectorConfig(any(), any(), isNull(), anyBoolean(), followerCallbackCaptor.capture());
// Leader will reply // 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); Herder.Created<ConnectorInfo> leaderAnswer = new Herder.Created<>(true, connectorInfo);
ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> leaderCallbackCaptor = ArgumentCaptor.forClass(Callback.class); ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> leaderCallbackCaptor = ArgumentCaptor.forClass(Callback.class);
doAnswer(invocation -> { 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.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.Properties; import java.util.Properties;
@ -98,10 +98,10 @@ public class SinkConnectorsIntegrationTest {
public void testEagerConsumerPartitionAssignment() throws Exception { public void testEagerConsumerPartitionAssignment() throws Exception {
final String topic1 = "topic1", topic2 = "topic2", topic3 = "topic3"; 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 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)); 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( connectorProps.put(
CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + PARTITION_ASSIGNMENT_STRATEGY_CONFIG, CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
RoundRobinAssignor.class.getName()); RoundRobinAssignor.class.getName());
@ -205,7 +205,7 @@ public class SinkConnectorsIntegrationTest {
public void testCooperativeConsumerPartitionAssignment() throws Exception { public void testCooperativeConsumerPartitionAssignment() throws Exception {
final String topic1 = "topic1", topic2 = "topic2", topic3 = "topic3"; 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 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)); Map<String, String> connectorProps = baseSinkConnectorProps(String.join(",", topics));
connectorProps.put( 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.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Properties; import java.util.Properties;
@ -150,7 +149,7 @@ public class SourceConnectorsIntegrationTest {
// start the clusters // start the clusters
connect.start(); 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().assertTopicsExist(BAR_TOPIC);
connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR, 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -38,7 +37,6 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;
import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.Response;
@ -102,7 +100,7 @@ public class StandaloneWorkerIntegrationTest {
StandaloneWorkerIntegrationTest::isModified StandaloneWorkerIntegrationTest::isModified
); );
assertEquals( assertEquals(
Collections.emptyMap(), Map.of(),
invalidModifiedLoggers, invalidModifiedLoggers,
"No loggers should have a non-null last-modified timestamp" "No loggers should have a non-null last-modified timestamp"
); );
@ -155,9 +153,9 @@ public class StandaloneWorkerIntegrationTest {
assertTrue(affectedLoggers.contains(namespace)); assertTrue(affectedLoggers.contains(namespace));
List<String> invalidAffectedLoggers = affectedLoggers.stream() List<String> invalidAffectedLoggers = affectedLoggers.stream()
.filter(l -> !l.startsWith(namespace)) .filter(l -> !l.startsWith(namespace))
.collect(Collectors.toList()); .toList();
assertEquals( assertEquals(
Collections.emptyList(), List.of(),
invalidAffectedLoggers, invalidAffectedLoggers,
"No loggers outside the namespace '" + namespace "No loggers outside the namespace '" + namespace
+ "' should have been included in the response for a request to modify that namespace" + "' should have been included in the response for a request to modify that namespace"
@ -188,7 +186,7 @@ public class StandaloneWorkerIntegrationTest {
) )
); );
assertEquals( assertEquals(
Collections.emptyMap(), Map.of(),
invalidAffectedLoggerLevels, invalidAffectedLoggerLevels,
"At least one logger in the affected namespace '" + namespace "At least one logger in the affected namespace '" + namespace
+ "' does not have the expected level of '" + level + "' 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()); Set<String> droppedLoggers = Utils.diff(HashSet::new, initialLevels.keySet(), newLevels.keySet());
assertEquals( assertEquals(
Collections.emptySet(), Set.of(),
droppedLoggers, droppedLoggers,
"At least one logger was present in the listing of all loggers " "At least one logger was present in the listing of all loggers "
+ "before the logging level for namespace '" + namespace + "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())) e -> !hasNamespace(e, namespace) && !e.getValue().equals(initialLevels.get(e.getKey()))
); );
assertEquals( assertEquals(
Collections.emptyMap(), Map.of(),
invalidUnaffectedLoggerLevels, invalidUnaffectedLoggerLevels,
"At least one logger outside of the affected namespace '" + namespace "At least one logger outside of the affected namespace '" + namespace
+ "' has a different logging level or last-modified timestamp than it did " + "' has a different logging level or last-modified timestamp than it did "
@ -256,8 +254,8 @@ public class StandaloneWorkerIntegrationTest {
CONNECTOR_NAME, CONNECTOR_NAME,
"Connector was not created in a stopped state" "Connector was not created in a stopped state"
); );
assertEquals(Collections.emptyList(), connect.connectorInfo(CONNECTOR_NAME).tasks()); assertEquals(List.of(), connect.connectorInfo(CONNECTOR_NAME).tasks());
assertEquals(Collections.emptyList(), connect.taskConfigs(CONNECTOR_NAME)); assertEquals(List.of(), connect.taskConfigs(CONNECTOR_NAME));
// Verify that a connector created in the STOPPED state can be resumed successfully // Verify that a connector created in the STOPPED state can be resumed successfully
connect.resumeConnector(CONNECTOR_NAME); 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.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
@ -92,7 +91,7 @@ public class StartAndStopLatchTest {
@Test @Test
public void shouldReturnFalseWhenAwaitingForDependentLatchToComplete() throws Throwable { public void shouldReturnFalseWhenAwaitingForDependentLatchToComplete() throws Throwable {
StartAndStopLatch depLatch = new StartAndStopLatch(1, 1, this::complete, null, clock); 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); latch = new StartAndStopLatch(1, 1, this::complete, dependents, clock);
future = asyncAwait(100); future = asyncAwait(100);
@ -106,7 +105,7 @@ public class StartAndStopLatchTest {
@Test @Test
public void shouldReturnTrueWhenAwaitingForStartAndStopAndDependentLatch() throws Throwable { public void shouldReturnTrueWhenAwaitingForStartAndStopAndDependentLatch() throws Throwable {
StartAndStopLatch depLatch = new StartAndStopLatch(1, 1, this::complete, null, clock); 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); latch = new StartAndStopLatch(1, 1, this::complete, dependents, clock);
future = asyncAwait(100); future = asyncAwait(100);

View File

@ -33,13 +33,11 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.LongStream; import java.util.stream.LongStream;
/** /**
@ -190,7 +188,7 @@ public class TestableSourceConnector extends SampleSourceConnector {
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId); taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
Map<String, Object> offset = Optional.ofNullable( Map<String, Object> offset = Optional.ofNullable(
context.offsetStorageReader().offset(sourcePartition(taskId))) context.offsetStorageReader().offset(sourcePartition(taskId)))
.orElse(Collections.emptyMap()); .orElse(Map.of());
startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L); startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L);
seqno = startingSeqno; seqno = startingSeqno;
log.info("Started {} task {} with properties {}", this.getClass().getSimpleName(), taskId, props); log.info("Started {} task {} with properties {}", this.getClass().getSimpleName(), taskId, props);
@ -235,7 +233,7 @@ public class TestableSourceConnector extends SampleSourceConnector {
maybeDefineTransactionBoundary(record); maybeDefineTransactionBoundary(record);
return record; return record;
}) })
.collect(Collectors.toList()); .toList();
} }
return null; return null;
} }
@ -295,10 +293,10 @@ public class TestableSourceConnector extends SampleSourceConnector {
} }
public static Map<String, Object> sourcePartition(String taskId) { 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) { 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.Properties;
import java.util.concurrent.TimeUnit; 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.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.PREDICATES_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.PREDICATES_CONFIG;
@ -173,7 +172,7 @@ public class TransformationIntegrationTest {
connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS); connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS);
// Assert that we didn't see any baz // 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); assertObservedRecords(observedRecords, expectedRecordCounts);
// delete connector // delete connector
@ -253,7 +252,7 @@ public class TransformationIntegrationTest {
// wait for the connector tasks to commit all records. // wait for the connector tasks to commit all records.
connectorHandle.awaitCommits(RECORD_TRANSFER_DURATION_MS); 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); assertObservedRecords(observedRecords, expectedRecordCounts);
// delete connector // delete connector

View File

@ -73,9 +73,7 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
@ -156,27 +154,27 @@ public class AbstractHerderTest {
private static final ClusterConfigState SNAPSHOT = new ClusterConfigState( private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(
1, 1,
null, null,
Collections.singletonMap(CONN1, 3), Map.of(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG), Map.of(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED), Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP, TASK_CONFIGS_MAP,
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Map.of(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Collections.emptySet(), Set.of(),
Collections.emptySet()); Set.of());
private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState( private static final ClusterConfigState SNAPSHOT_NO_TASKS = new ClusterConfigState(
1, 1,
null, null,
Collections.singletonMap(CONN1, 3), Map.of(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG), Map.of(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED), Map.of(CONN1, TargetState.STARTED),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)), Map.of(CONN1, new AppliedConnectorConfig(CONN1_CONFIG)),
Collections.emptySet(), Set.of(),
Collections.emptySet()); Set.of());
private final String workerId = "workerId"; private final String workerId = "workerId";
private final String kafkaClusterId = "I4ZmrWqfT2e-upky_4fdPA"; private final String kafkaClusterId = "I4ZmrWqfT2e-upky_4fdPA";
@ -198,7 +196,7 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder(); AbstractHerder herder = testHerder();
when(configStore.snapshot()).thenReturn(SNAPSHOT); when(configStore.snapshot()).thenReturn(SNAPSHOT);
assertEquals(Collections.singleton(CONN1), new HashSet<>(herder.connectors())); assertEquals(Set.of(CONN1), Set.copyOf(herder.connectors()));
} }
@Test @Test
@ -220,7 +218,7 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder(); AbstractHerder herder = testHerder();
when(herder.rawConfig(connectorName)).thenReturn(Collections.singletonMap( when(herder.rawConfig(connectorName)).thenReturn(Map.of(
ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName() ConnectorConfig.CONNECTOR_CLASS_CONFIG, SampleSourceConnector.class.getName()
)); ));
@ -228,7 +226,7 @@ public class AbstractHerderTest {
.thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation)); .thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation));
when(statusStore.getAll(connectorName)) when(statusStore.getAll(connectorName))
.thenReturn(Collections.singletonList( .thenReturn(List.of(
new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation))); new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation)));
ConnectorStateInfo state = herder.connectorStatus(connectorName); ConnectorStateInfo state = herder.connectorStatus(connectorName);
@ -255,13 +253,13 @@ public class AbstractHerderTest {
AbstractHerder herder = testHerder(); AbstractHerder herder = testHerder();
when(herder.rawConfig(connectorName)) when(herder.rawConfig(connectorName))
.thenReturn(Collections.singletonMap(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "missing")); .thenReturn(Map.of(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "missing"));
when(statusStore.get(connectorName)) when(statusStore.get(connectorName))
.thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation)); .thenReturn(new ConnectorStatus(connectorName, AbstractStatus.State.RUNNING, workerId, generation));
when(statusStore.getAll(connectorName)) when(statusStore.getAll(connectorName))
.thenReturn(Collections.singletonList( .thenReturn(List.of(
new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation))); new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation)));
ConnectorStateInfo state = herder.connectorStatus(connectorName); ConnectorStateInfo state = herder.connectorStatus(connectorName);
@ -292,7 +290,7 @@ public class AbstractHerderTest {
assertEquals(CONN1, info.name()); assertEquals(CONN1, info.name());
assertEquals(CONN1_CONFIG, info.config()); 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()); assertEquals(ConnectorType.SOURCE, info.type());
} }
@ -332,7 +330,7 @@ public class AbstractHerderTest {
assertEquals(CONN1, info.name()); assertEquals(CONN1, info.name());
assertEquals(CONN1_CONFIG, info.config()); 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()); assertEquals(ConnectorType.UNKNOWN, info.type());
} }
@ -468,8 +466,8 @@ public class AbstractHerderTest {
public void testConfigValidationEmptyConfig() { public void testConfigValidationEmptyConfig() {
AbstractHerder herder = createConfigValidationHerder(SampleSourceConnector.class, noneConnectorClientConfigOverridePolicy, 0); AbstractHerder herder = createConfigValidationHerder(SampleSourceConnector.class, noneConnectorClientConfigOverridePolicy, 0);
assertThrows(BadRequestException.class, () -> herder.validateConnectorConfig(Collections.emptyMap(), s -> null, false)); assertThrows(BadRequestException.class, () -> herder.validateConnectorConfig(Map.of(), s -> null, false));
verify(transformer).transform(Collections.emptyMap()); verify(transformer).transform(Map.of());
assertEquals(worker.getPlugins(), plugins); assertEquals(worker.getPlugins(), plugins);
} }
@ -478,13 +476,13 @@ public class AbstractHerderTest {
final Class<? extends Connector> connectorClass = SampleSourceConnector.class; final Class<? extends Connector> connectorClass = SampleSourceConnector.class;
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy); 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); 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 // 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. // the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name()); 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, ConnectorConfig.PREDICATES_GROUP, ConnectorConfig.ERROR_GROUP,
SourceConnectorConfig.TOPIC_CREATION_GROUP, SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_GROUP, SourceConnectorConfig.TOPIC_CREATION_GROUP, SourceConnectorConfig.EXACTLY_ONCE_SUPPORT_GROUP,
SourceConnectorConfig.OFFSETS_TOPIC_GROUP), result.groups()); SourceConnectorConfig.OFFSETS_TOPIC_GROUP), result.groups());
@ -572,7 +570,7 @@ public class AbstractHerderTest {
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy); AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy);
// 2 transform aliases defined -> 2 plugin lookups // 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()); 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 // 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. // the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name()); assertEquals(connectorClass.getName(), result.name());
// Each transform also gets its own group // Each transform also gets its own group
List<String> expectedGroups = Arrays.asList( List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP, ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP, ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP, ConnectorConfig.PREDICATES_GROUP,
@ -626,8 +624,8 @@ public class AbstractHerderTest {
final Class<? extends Connector> connectorClass = SampleSourceConnector.class; final Class<? extends Connector> connectorClass = SampleSourceConnector.class;
AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy); AbstractHerder herder = createConfigValidationHerder(connectorClass, noneConnectorClientConfigOverridePolicy);
Mockito.lenient().when(plugins.transformations()).thenReturn(Collections.singleton(transformationPluginDesc())); Mockito.lenient().when(plugins.transformations()).thenReturn(Set.of(transformationPluginDesc()));
Mockito.lenient().when(plugins.predicates()).thenReturn(Collections.singleton(predicatePluginDesc())); 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(SampleTransformation.class.getName(), null, classLoader)).thenReturn(new SampleTransformation());
Mockito.lenient().when(plugins.newPlugin(SamplePredicate.class.getName(), null, classLoader)).thenReturn(new SamplePredicate()); 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. // the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(connectorClass.getName(), result.name()); assertEquals(connectorClass.getName(), result.name());
// Each transform also gets its own group // Each transform also gets its own group
List<String> expectedGroups = Arrays.asList( List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP, ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP, ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP, ConnectorConfig.PREDICATES_GROUP,
@ -716,7 +714,7 @@ public class AbstractHerderTest {
// the config fields for SourceConnectorConfig, but we expect these to change rarely. // the config fields for SourceConnectorConfig, but we expect these to change rarely.
assertEquals(SampleSourceConnector.class.getName(), result.name()); assertEquals(SampleSourceConnector.class.getName(), result.name());
// Each transform also gets its own group // Each transform also gets its own group
List<String> expectedGroups = Arrays.asList( List<String> expectedGroups = List.of(
ConnectorConfig.COMMON_GROUP, ConnectorConfig.COMMON_GROUP,
ConnectorConfig.TRANSFORMS_GROUP, ConnectorConfig.TRANSFORMS_GROUP,
ConnectorConfig.PREDICATES_GROUP, ConnectorConfig.PREDICATES_GROUP,
@ -891,7 +889,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B"); addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C"); 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<>(); List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1"); addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1"); addValue(values, "config.b1", "value.b1");
@ -922,7 +920,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B"); addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C"); 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<>(); List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1"); addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1"); addValue(values, "config.b1", "value.b1");
@ -953,7 +951,7 @@ public class AbstractHerderTest {
addConfigKey(keys, "config.b2", "group B"); addConfigKey(keys, "config.b2", "group B");
addConfigKey(keys, "config.c1", "group C"); 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<>(); List<ConfigValue> values = new ArrayList<>();
addValue(values, "config.a1", "value.a1"); addValue(values, "config.a1", "value.a1");
addValue(values, "config.b1", "value.b1"); addValue(values, "config.b1", "value.b1");
@ -1143,7 +1141,7 @@ public class AbstractHerderTest {
when(worker.getPlugins()).thenReturn(plugins); when(worker.getPlugins()).thenReturn(plugins);
when(plugins.newConnector(anyString(), any())).thenThrow(new ConnectException("No class found")); when(plugins.newConnector(anyString(), any())).thenThrow(new ConnectException("No class found"));
AbstractHerder herder = testHerder(); 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 @Test
@ -1155,7 +1153,7 @@ public class AbstractHerderTest {
@Test @Test
public void testGetConnectorTypeWithEmptyConfig() { public void testGetConnectorTypeWithEmptyConfig() {
AbstractHerder herder = testHerder(); AbstractHerder herder = testHerder();
assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Collections.emptyMap())); assertEquals(ConnectorType.UNKNOWN, herder.connectorType(Map.of()));
} }
@Test @Test
@ -1170,9 +1168,9 @@ public class AbstractHerderTest {
@Test @Test
public void testConnectorOffsets() throws Exception { public void testConnectorOffsets() throws Exception {
ConnectorOffsets offsets = new ConnectorOffsets(Arrays.asList( ConnectorOffsets offsets = new ConnectorOffsets(List.of(
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")), new ConnectorOffset(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue")) new ConnectorOffset(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"))
)); ));
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ArgumentCaptor<Callback<ConnectorOffsets>> workerCallback = ArgumentCaptor.forClass(Callback.class); ArgumentCaptor<Callback<ConnectorOffsets>> workerCallback = ArgumentCaptor.forClass(Callback.class);
@ -1203,7 +1201,7 @@ public class AbstractHerderTest {
when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIG.size()); when(snapshot.taskCount(CONN1)).thenReturn(TASK_CONFIG.size());
List<Map<String, String>> alteredTaskConfigs = new ArrayList<>(TASK_CONFIGS); 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 // Last task config is different; should report a change
assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, alteredTaskConfigs)); assertTrue(AbstractHerder.taskConfigsChanged(snapshot, CONN1, alteredTaskConfigs));
@ -1220,15 +1218,15 @@ public class AbstractHerderTest {
ClusterConfigState snapshotWithNoAppliedConfig = new ClusterConfigState( ClusterConfigState snapshotWithNoAppliedConfig = new ClusterConfigState(
1, 1,
null, null,
Collections.singletonMap(CONN1, 3), Map.of(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG), Map.of(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED), Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP, TASK_CONFIGS_MAP,
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet() Set.of()
); );
assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithNoAppliedConfig, CONN1, TASK_CONFIGS)); assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithNoAppliedConfig, CONN1, TASK_CONFIGS));
@ -1238,15 +1236,15 @@ public class AbstractHerderTest {
ClusterConfigState snapshotWithDifferentAppliedConfig = new ClusterConfigState( ClusterConfigState snapshotWithDifferentAppliedConfig = new ClusterConfigState(
1, 1,
null, null,
Collections.singletonMap(CONN1, 3), Map.of(CONN1, 3),
Collections.singletonMap(CONN1, CONN1_CONFIG), Map.of(CONN1, CONN1_CONFIG),
Collections.singletonMap(CONN1, TargetState.STARTED), Map.of(CONN1, TargetState.STARTED),
TASK_CONFIGS_MAP, TASK_CONFIGS_MAP,
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONN1, new AppliedConnectorConfig(appliedConfig)), Map.of(CONN1, new AppliedConnectorConfig(appliedConfig)),
Collections.emptySet(), Set.of(),
Collections.emptySet() Set.of()
); );
assertTrue(AbstractHerder.taskConfigsChanged(snapshotWithDifferentAppliedConfig, CONN1, TASK_CONFIGS)); 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) { protected void addConfigKey(Map<String, ConfigDef.ConfigKey> keys, String name, String group) {
ConfigDef configDef = new ConfigDef().define(name, ConfigDef.Type.STRING, null, null, ConfigDef configDef = new ConfigDef().define(name, ConfigDef.Type.STRING, null, null,
ConfigDef.Importance.HIGH, "doc", group, 10, 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()); keys.putAll(configDef.configKeys());
} }
protected void addValue(List<ConfigValue> values, String name, String value, String... errors) { 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) { protected void assertInfoKey(ConfigInfos infos, String name, String group) {
@ -1277,7 +1275,7 @@ public class AbstractHerderTest {
ConfigValueInfo info = findInfo(infos, name).configValue(); ConfigValueInfo info = findInfo(infos, name).configValue();
assertEquals(name, info.name()); assertEquals(name, info.name());
assertEquals(value, info.value()); assertEquals(value, info.value());
assertEquals(Arrays.asList(errors), info.errors()); assertEquals(List.of(errors), info.errors());
} }
protected ConfigInfo findInfo(ConfigInfos infos, String name) { protected ConfigInfo findInfo(ConfigInfos infos, String name) {
@ -1293,7 +1291,7 @@ public class AbstractHerderTest {
} }
private void testConfigProviderRegex(String rawConnConfig, boolean expected) { 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"); boolean actual = !keys.isEmpty() && keys.contains("key");
assertEquals(expected, actual, String.format("%s should have matched regex", rawConnConfig)); 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.ByteBuffer;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -121,8 +119,8 @@ public class AbstractWorkerSourceTaskTest {
private static final String TOPIC = "topic"; private static final String TOPIC = "topic";
private static final String OTHER_TOPIC = "other-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, byte[]> PARTITION = Map.of("key", "partition".getBytes());
private static final Map<String, Integer> OFFSET = Collections.singletonMap("key", 12); private static final Map<String, Integer> OFFSET = Map.of("key", 12);
// Connect-format data // Connect-format data
private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA;
@ -253,7 +251,7 @@ public class AbstractWorkerSourceTaskTest {
createWorkerTask(); createWorkerTask();
// Can just use the same record for key and value // 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) new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)
); );
@ -282,7 +280,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain(); expectApplyTransformationChain();
expectTopicCreation(TOPIC); expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList( workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
); );
workerTask.sendRecords(); workerTask.sendRecords();
@ -302,7 +300,7 @@ public class AbstractWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC); expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC);
expectApplyTransformationChain(); expectApplyTransformationChain();
workerTask.toSend = Collections.singletonList( workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
); );
assertThrows(InvalidRecordException.class, workerTask::sendRecords); assertThrows(InvalidRecordException.class, workerTask::sendRecords);
@ -319,7 +317,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain(); expectApplyTransformationChain();
expectTopicCreation(TOPIC); expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList( workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp)
); );
workerTask.sendRecords(); workerTask.sendRecords();
@ -345,7 +343,7 @@ public class AbstractWorkerSourceTaskTest {
expectApplyTransformationChain(); expectApplyTransformationChain();
expectTopicCreation(TOPIC); expectTopicCreation(TOPIC);
workerTask.toSend = Collections.singletonList( workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD,
null, connectHeaders) null, connectHeaders)
); );
@ -367,7 +365,7 @@ public class AbstractWorkerSourceTaskTest {
SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders(); SampleConverterWithHeaders testConverter = new SampleConverterWithHeaders();
createWorkerTask(stringConverter, testConverter, stringConverter, RetryWithToleranceOperatorTest.noneOperator(), createWorkerTask(stringConverter, testConverter, stringConverter, RetryWithToleranceOperatorTest.noneOperator(),
Collections::emptyList, transformationChain); List::of, transformationChain);
expectSendRecord(null); expectSendRecord(null);
expectApplyTransformationChain(); expectApplyTransformationChain();
@ -383,7 +381,7 @@ public class AbstractWorkerSourceTaskTest {
org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders() org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders()
.addString("encoding", encodingB); .addString("encoding", encodingB);
workerTask.toSend = Arrays.asList( workerTask.toSend = List.of(
new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a", new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "a",
Schema.STRING_SCHEMA, stringA, null, headersA), Schema.STRING_SCHEMA, stringA, null, headersA),
new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b", new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b",
@ -426,13 +424,13 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC); expectPreliminaryCalls(TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc)); when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
expectSendRecord(emptyHeaders()); expectSendRecord(emptyHeaders());
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords(); workerTask.sendRecords();
verifySendRecord(2); verifySendRecord(2);
@ -452,11 +450,11 @@ public class AbstractWorkerSourceTaskTest {
when(admin.describeTopics(TOPIC)) when(admin.describeTopics(TOPIC))
.thenThrow(new RetriableException(new TimeoutException("timeout"))) .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(); workerTask.sendRecords();
assertEquals(Arrays.asList(record1, record2), workerTask.toSend); assertEquals(List.of(record1, record2), workerTask.toSend);
verify(admin, never()).createOrFindTopics(any(NewTopic.class)); verify(admin, never()).createOrFindTopics(any(NewTopic.class));
verifyNoMoreInteractions(admin); verifyNoMoreInteractions(admin);
@ -477,16 +475,16 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC); expectPreliminaryCalls(TOPIC);
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap()); when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))) when(admin.createOrFindTopics(any(NewTopic.class)))
// First call to create the topic times out // First call to create the topic times out
.thenThrow(new RetriableException(new TimeoutException("timeout"))) .thenThrow(new RetriableException(new TimeoutException("timeout")))
// Next attempt succeeds // Next attempt succeeds
.thenReturn(createdTopic(TOPIC)); .thenReturn(createdTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords(); workerTask.sendRecords();
assertEquals(Arrays.asList(record1, record2), workerTask.toSend); assertEquals(List.of(record1, record2), workerTask.toSend);
// Next they all succeed // Next they all succeed
workerTask.sendRecords(); workerTask.sendRecords();
@ -509,9 +507,9 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(OTHER_TOPIC); expectPreliminaryCalls(OTHER_TOPIC);
when(admin.describeTopics(anyString())) when(admin.describeTopics(anyString()))
.thenReturn(Collections.emptyMap()) .thenReturn(Map.of())
.thenThrow(new RetriableException(new TimeoutException("timeout"))) .thenThrow(new RetriableException(new TimeoutException("timeout")))
.thenReturn(Collections.emptyMap()); .thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenAnswer( when(admin.createOrFindTopics(any(NewTopic.class))).thenAnswer(
(Answer<TopicAdmin.TopicCreationResponse>) invocation -> { (Answer<TopicAdmin.TopicCreationResponse>) invocation -> {
NewTopic newTopic = invocation.getArgument(0); 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 // 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(); workerTask.sendRecords();
assertEquals(Collections.singletonList(record3), workerTask.toSend); assertEquals(List.of(record3), workerTask.toSend);
// Next they all succeed // Next they all succeed
workerTask.sendRecords(); workerTask.sendRecords();
@ -532,10 +530,10 @@ public class AbstractWorkerSourceTaskTest {
ArgumentCaptor<NewTopic> newTopicCaptor = ArgumentCaptor.forClass(NewTopic.class); ArgumentCaptor<NewTopic> newTopicCaptor = ArgumentCaptor.forClass(NewTopic.class);
verify(admin, times(2)).createOrFindTopics(newTopicCaptor.capture()); verify(admin, times(2)).createOrFindTopics(newTopicCaptor.capture());
assertEquals(Arrays.asList(TOPIC, OTHER_TOPIC), newTopicCaptor.getAllValues() assertEquals(List.of(TOPIC, OTHER_TOPIC), newTopicCaptor.getAllValues()
.stream() .stream()
.map(NewTopic::name) .map(NewTopic::name)
.collect(Collectors.toList())); .toList());
} }
@Test @Test
@ -550,16 +548,16 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC); expectPreliminaryCalls(TOPIC);
expectPreliminaryCalls(OTHER_TOPIC); expectPreliminaryCalls(OTHER_TOPIC);
when(admin.describeTopics(anyString())).thenReturn(Collections.emptyMap()); when(admin.describeTopics(anyString())).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))) when(admin.createOrFindTopics(any(NewTopic.class)))
.thenReturn(createdTopic(TOPIC)) .thenReturn(createdTopic(TOPIC))
.thenThrow(new RetriableException(new TimeoutException("timeout"))) .thenThrow(new RetriableException(new TimeoutException("timeout")))
.thenReturn(createdTopic(OTHER_TOPIC)); .thenReturn(createdTopic(OTHER_TOPIC));
// Try to send 3, make first pass, second fail. Should save last record // 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(); 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 verifyTopicCreation(2, TOPIC, OTHER_TOPIC); // Second call to createOrFindTopics will throw
// Next they all succeed // Next they all succeed
@ -581,7 +579,7 @@ public class AbstractWorkerSourceTaskTest {
new ConnectException(new TopicAuthorizationException("unauthorized")) new ConnectException(new TopicAuthorizationException("unauthorized"))
); );
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords); 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); SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD);
expectPreliminaryCalls(TOPIC); expectPreliminaryCalls(TOPIC);
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap()); when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenThrow( when(admin.createOrFindTopics(any(NewTopic.class))).thenThrow(
new ConnectException(new TopicAuthorizationException("unauthorized")) new ConnectException(new TopicAuthorizationException("unauthorized"))
); );
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
assertThrows(ConnectException.class, workerTask::sendRecords); assertThrows(ConnectException.class, workerTask::sendRecords);
verify(admin).createOrFindTopics(any()); verify(admin).createOrFindTopics(any());
@ -614,10 +612,10 @@ public class AbstractWorkerSourceTaskTest {
expectPreliminaryCalls(TOPIC); 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); 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); assertThrows(ConnectException.class, workerTask::sendRecords);
verify(admin).createOrFindTopics(any()); verify(admin).createOrFindTopics(any());
@ -634,10 +632,10 @@ public class AbstractWorkerSourceTaskTest {
expectSendRecord(emptyHeaders()); expectSendRecord(emptyHeaders());
expectApplyTransformationChain(); expectApplyTransformationChain();
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap()); when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(foundTopic(TOPIC)); when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(foundTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords(); workerTask.sendRecords();
ArgumentCaptor<ProducerRecord<byte[], byte[]>> sent = verifySendRecord(2); ArgumentCaptor<ProducerRecord<byte[], byte[]>> sent = verifySendRecord(2);
@ -659,10 +657,10 @@ public class AbstractWorkerSourceTaskTest {
expectSendRecord(emptyHeaders()); expectSendRecord(emptyHeaders());
expectApplyTransformationChain(); expectApplyTransformationChain();
when(admin.describeTopics(TOPIC)).thenReturn(Collections.emptyMap()); when(admin.describeTopics(TOPIC)).thenReturn(Map.of());
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(createdTopic(TOPIC)); when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(createdTopic(TOPIC));
workerTask.toSend = Arrays.asList(record1, record2); workerTask.toSend = List.of(record1, record2);
workerTask.sendRecords(); workerTask.sendRecords();
ArgumentCaptor<ProducerRecord<byte[], byte[]>> sent = verifySendRecord(2); 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(record2))).thenReturn(null);
when(transformationChain.apply(any(), eq(record3))).thenReturn(record3); when(transformationChain.apply(any(), eq(record3))).thenReturn(record3);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc)); when(admin.describeTopics(TOPIC)).thenReturn(Map.of(TOPIC, topicDesc));
when(producer.send(any(), any())).thenThrow(new RetriableException("Retriable exception")).thenReturn(null); 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 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 // 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); expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc)); 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 // The transformation errored out so the error should be re-raised by sendRecords with error tolerance None
Exception exception = assertThrows(ConnectException.class, workerTask::sendRecords); Exception exception = assertThrows(ConnectException.class, workerTask::sendRecords);
@ -752,7 +750,7 @@ public class AbstractWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(emptyHeaders(), TOPIC); 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 // The transformation errored out so the error should be ignored & the record skipped with error tolerance all
assertTrue(workerTask.sendRecords()); assertTrue(workerTask.sendRecords());
@ -778,11 +776,11 @@ public class AbstractWorkerSourceTaskTest {
// When we try to convert the key/value of each record, throw an exception // When we try to convert the key/value of each record, throw an exception
throwExceptionWhenConvertKey(emptyHeaders(), TOPIC); throwExceptionWhenConvertKey(emptyHeaders(), TOPIC);
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, List.of(), List.of());
TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); TopicDescription topicDesc = new TopicDescription(TOPIC, false, List.of(topicPartitionInfo));
when(admin.describeTopics(TOPIC)).thenReturn(Collections.singletonMap(TOPIC, topicDesc)); 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 // Send records should fail when errors.tolerance is none and the conversion call fails
Exception exception = assertThrows(ConnectException.class, workerTask::sendRecords); 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 // When we try to convert the key/value of each record, throw an exception
throwExceptionWhenConvertKey(emptyHeaders(), TOPIC); 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 // With errors.tolerance to all, the failed conversion should simply skip the record, and record successful batch
assertTrue(workerTask.sendRecords()); assertTrue(workerTask.sendRecords());
@ -883,15 +881,15 @@ public class AbstractWorkerSourceTaskTest {
@SuppressWarnings("SameParameterValue") @SuppressWarnings("SameParameterValue")
private TopicAdmin.TopicCreationResponse createdTopic(String topic) { private TopicAdmin.TopicCreationResponse createdTopic(String topic) {
Set<String> created = Collections.singleton(topic); Set<String> created = Set.of(topic);
Set<String> existing = Collections.emptySet(); Set<String> existing = Set.of();
return new TopicAdmin.TopicCreationResponse(created, existing); return new TopicAdmin.TopicCreationResponse(created, existing);
} }
@SuppressWarnings("SameParameterValue") @SuppressWarnings("SameParameterValue")
private TopicAdmin.TopicCreationResponse foundTopic(String topic) { private TopicAdmin.TopicCreationResponse foundTopic(String topic) {
Set<String> created = Collections.emptySet(); Set<String> created = Set.of();
Set<String> existing = Collections.singleton(topic); Set<String> existing = Set.of(topic);
return new TopicAdmin.TopicCreationResponse(created, existing); return new TopicAdmin.TopicCreationResponse(created, existing);
} }
@ -945,13 +943,13 @@ public class AbstractWorkerSourceTaskTest {
} }
private void createWorkerTask(TransformationChain transformationChain, RetryWithToleranceOperator toleranceOperator) { private void createWorkerTask(TransformationChain transformationChain, RetryWithToleranceOperator toleranceOperator) {
createWorkerTask(keyConverter, valueConverter, headerConverter, toleranceOperator, Collections::emptyList, createWorkerTask(keyConverter, valueConverter, headerConverter, toleranceOperator, List::of,
transformationChain); transformationChain);
} }
private void createWorkerTask() { private void createWorkerTask() {
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, 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 org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; 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<>()) { public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap<>()) {
@Override @Override
public Set<PluginDesc<Transformation<?>>> transformations() { 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); Plugins mockPlugins = mock(Plugins.class);
when(mockPlugins.newPlugin(HasDuplicateConfigTransformation.class.getName(), when(mockPlugins.newPlugin(HasDuplicateConfigTransformation.class.getName(),
null, (ClassLoader) null)).thenReturn(new HasDuplicateConfigTransformation()); 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); ConfigDef def = ConnectorConfig.enrich(mockPlugins, new ConfigDef(), props, false);
assertEnrichedConfigDef(def, prefix, HasDuplicateConfigTransformation.MUST_EXIST_KEY, ConfigDef.Type.BOOLEAN); assertEnrichedConfigDef(def, prefix, HasDuplicateConfigTransformation.MUST_EXIST_KEY, ConfigDef.Type.BOOLEAN);
assertEnrichedConfigDef(def, prefix, TransformationStage.PREDICATE_CONFIG, ConfigDef.Type.STRING); 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.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; 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.common.utils.Time.SYSTEM;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG; 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.CONNECTOR_CLASS_CONFIG;
@ -226,7 +223,7 @@ public class ErrorHandlingTaskTest {
LogReporter<ConsumerRecord<byte[], byte[]>> reporter = new LogReporter.Sink(taskId, connConfig(reportProps), errorHandlingMetrics); LogReporter<ConsumerRecord<byte[], byte[]>> reporter = new LogReporter.Sink(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator = operator(); RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator = operator();
createSinkTask(initialState, retryWithToleranceOperator, singletonList(reporter)); createSinkTask(initialState, retryWithToleranceOperator, List.of(reporter));
// valid json // valid json
ConsumerRecord<byte[], byte[]> record1 = new ConsumerRecord<>( ConsumerRecord<byte[], byte[]> record1 = new ConsumerRecord<>(
@ -278,14 +275,14 @@ public class ErrorHandlingTaskTest {
LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics); LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator(); RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator();
createSourceTask(initialState, retryWithToleranceOperator, singletonList(reporter)); createSourceTask(initialState, retryWithToleranceOperator, List.of(reporter));
// valid json // valid json
Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build();
Struct struct1 = new Struct(valSchema).put("val", 1234); 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); 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()) when(workerSourceTask.isStopping())
.thenReturn(false) .thenReturn(false)
@ -295,8 +292,8 @@ public class ErrorHandlingTaskTest {
doReturn(true).when(workerSourceTask).commitOffsets(); doReturn(true).when(workerSourceTask).commitOffsets();
when(sourceTask.poll()) when(sourceTask.poll())
.thenReturn(singletonList(record1)) .thenReturn(List.of(record1))
.thenReturn(singletonList(record2)); .thenReturn(List.of(record2));
expectTopicCreation(TOPIC); expectTopicCreation(TOPIC);
@ -340,14 +337,14 @@ public class ErrorHandlingTaskTest {
LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics); LogReporter<SourceRecord> reporter = new LogReporter.Source(taskId, connConfig(reportProps), errorHandlingMetrics);
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator(); RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = operator();
createSourceTask(initialState, retryWithToleranceOperator, singletonList(reporter), badConverter()); createSourceTask(initialState, retryWithToleranceOperator, List.of(reporter), badConverter());
// valid json // valid json
Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build();
Struct struct1 = new Struct(valSchema).put("val", 1234); 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); 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()) when(workerSourceTask.isStopping())
.thenReturn(false) .thenReturn(false)
@ -357,8 +354,8 @@ public class ErrorHandlingTaskTest {
doReturn(true).when(workerSourceTask).commitOffsets(); doReturn(true).when(workerSourceTask).commitOffsets();
when(sourceTask.poll()) when(sourceTask.poll())
.thenReturn(singletonList(record1)) .thenReturn(List.of(record1))
.thenReturn(singletonList(record2)); .thenReturn(List.of(record2));
expectTopicCreation(TOPIC); expectTopicCreation(TOPIC);
workerSourceTask.initialize(TASK_CONFIG); workerSourceTask.initialize(TASK_CONFIG);
workerSourceTask.initializeAndStart(); workerSourceTask.initializeAndStart();
@ -392,7 +389,7 @@ public class ErrorHandlingTaskTest {
private void verifyInitializeSink() { private void verifyInitializeSink() {
verify(sinkTask).start(TASK_PROPS); verify(sinkTask).start(TASK_PROPS);
verify(sinkTask).initialize(any(WorkerSinkTaskContext.class)); verify(sinkTask).initialize(any(WorkerSinkTaskContext.class));
verify(consumer).subscribe(eq(singletonList(TOPIC)), verify(consumer).subscribe(eq(List.of(TOPIC)),
any(ConsumerRebalanceListener.class)); any(ConsumerRebalanceListener.class));
} }
@ -410,9 +407,9 @@ public class ErrorHandlingTaskTest {
private void expectTopicCreation(String topic) { private void expectTopicCreation(String topic) {
if (enableTopicCreation) { if (enableTopicCreation) {
when(admin.describeTopics(topic)).thenReturn(Collections.emptyMap()); when(admin.describeTopics(topic)).thenReturn(Map.of());
Set<String> created = Collections.singleton(topic); Set<String> created = Set.of(topic);
Set<String> existing = Collections.emptySet(); Set<String> existing = Set.of();
TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing); TopicAdmin.TopicCreationResponse response = new TopicAdmin.TopicCreationResponse(created, existing);
when(admin.createOrFindTopics(any(NewTopic.class))).thenReturn(response); 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"); Plugin<Transformation<SinkRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SinkRecord>(), taskId, "test");
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> sinkTransforms = 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> keyConverterPlugin = metrics.wrap(converter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(converter, taskId, false); Plugin<Converter> valueConverterPlugin = metrics.wrap(converter, taskId, false);
@ -463,7 +460,7 @@ public class ErrorHandlingTaskTest {
private void createSourceTask(TargetState initialState, RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator, private void createSourceTask(TargetState initialState, RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator,
List<ErrorReporter<SourceRecord>> errorReporters, Converter converter) { List<ErrorReporter<SourceRecord>> errorReporters, Converter converter) {
Plugin<Transformation<SourceRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SourceRecord>(), taskId, "test"); 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); new TransformationStage<>(transformationPlugin, "test", null, TestPlugins.noOpLoaderSwap())), retryWithToleranceOperator);
Plugin<Converter> keyConverterPlugin = metrics.wrap(converter, taskId, true); 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 org.mockito.verification.VerificationMode;
import java.time.Duration; import java.time.Duration;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -84,7 +82,6 @@ import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer; import java.util.function.Consumer;
import java.util.stream.Collectors; 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.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; 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.KEY_CONVERTER_CLASS_CONFIG;
@ -119,7 +116,7 @@ import static org.mockito.Mockito.when;
@MockitoSettings(strictness = Strictness.WARN) @MockitoSettings(strictness = Strictness.WARN)
public class ExactlyOnceWorkerSourceTaskTest { public class ExactlyOnceWorkerSourceTaskTest {
private static final String TOPIC = "topic"; 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); private static final Map<String, ?> OFFSET = offset(12);
// Connect-format data // Connect-format data
@ -169,7 +166,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
private static final SourceRecord SOURCE_RECORD_2 = private static final SourceRecord SOURCE_RECORD_2 =
new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, VALUE_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<CountDownLatch> pollLatch = new AtomicReference<>(new CountDownLatch(0));
private final AtomicReference<List<SourceRecord>> pollRecords = new AtomicReference<>(RECORDS); private final AtomicReference<List<SourceRecord>> pollRecords = new AtomicReference<>(RECORDS);
@ -267,7 +264,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
} }
private static Map<String, ?> offset(int n) { private static Map<String, ?> offset(int n) {
return Collections.singletonMap("key", n); return Map.of("key", n);
} }
private void createWorkerTask() { private void createWorkerTask() {
@ -285,7 +282,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin,
transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore, transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore,
config, clusterConfigState, metrics, errorHandlingMetrics, plugins.delegatingLoader(), time, RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore, 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 @ParameterizedTest
@ -296,7 +293,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
workerTask.removeMetrics(); workerTask.removeMetrics();
assertEquals(emptySet(), filterToTaskMetrics(metrics.metrics().metrics().keySet())); assertEquals(Set.of(), filterToTaskMetrics(metrics.metrics().metrics().keySet()));
} }
private Set<MetricName> filterToTaskMetrics(Set<MetricName> metricNames) { private Set<MetricName> filterToTaskMetrics(Set<MetricName> metricNames) {
@ -562,7 +559,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
createWorkerTask(); createWorkerTask();
// Make sure the task returns empty batches from poll before we start polling it // 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); when(offsetWriter.beginFlush()).thenReturn(false);
@ -956,7 +953,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
expectConvertHeadersAndKeyValue(); expectConvertHeadersAndKeyValue();
// We're trying to send three records // 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())); OngoingStubbing<Future<RecordMetadata>> producerSend = when(producer.send(any(), any()));
// The first one is sent successfully // The first one is sent successfully
producerSend = expectSuccessfulSend(producerSend); producerSend = expectSuccessfulSend(producerSend);
@ -966,7 +963,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
expectSuccessfulSend(producerSend); expectSuccessfulSend(producerSend);
assertFalse(workerTask.sendRecords()); assertFalse(workerTask.sendRecords());
assertEquals(Arrays.asList(record2, record3), workerTask.toSend); assertEquals(List.of(record2, record3), workerTask.toSend);
verify(producer).beginTransaction(); verify(producer).beginTransaction();
// When using poll-based transaction boundaries, we do not commit transactions while retrying delivery for a batch // When using poll-based transaction boundaries, we do not commit transactions while retrying delivery for a batch
verify(producer, never()).commitTransaction(); verify(producer, never()).commitTransaction();
@ -1001,7 +998,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
when(producer.send(any(), any())) when(producer.send(any(), any()))
.thenThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC))); .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); assertThrows(ConnectException.class, workerTask::sendRecords);
verify(producer).beginTransaction(); verify(producer).beginTransaction();
@ -1080,7 +1077,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
} }
private void awaitEmptyPolls(int minimum) { private void awaitEmptyPolls(int minimum) {
awaitPolls(minimum, Collections.emptyList()); awaitPolls(minimum, List.of());
} }
private void awaitPolls(int minimum) { private void awaitPolls(int minimum) {
@ -1168,8 +1165,8 @@ public class ExactlyOnceWorkerSourceTaskTest {
private void expectPossibleTopicCreation() { private void expectPossibleTopicCreation() {
if (config.topicCreationEnable()) { if (config.topicCreationEnable()) {
Set<String> created = Collections.singleton(TOPIC); Set<String> created = Set.of(TOPIC);
Set<String> existing = Collections.emptySet(); Set<String> existing = Set.of();
TopicAdmin.TopicCreationResponse creationResponse = new TopicAdmin.TopicCreationResponse(created, existing); TopicAdmin.TopicCreationResponse creationResponse = new TopicAdmin.TopicCreationResponse(created, existing);
when(admin.createOrFindTopics(any())).thenReturn(creationResponse); 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.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@ -49,7 +49,7 @@ public class InternalSinkRecordTest {
assertTrue(sinkRecord.headers().isEmpty()); assertTrue(sinkRecord.headers().isEmpty());
SinkRecord newRecord = internalSinkRecord.newRecord(TOPIC, 0, null, null, null, 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()); 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.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -72,7 +70,7 @@ public class MockLoggersTest {
Loggers loggers = new TestLoggers(root, a, b); Loggers loggers = new TestLoggers(root, a, b);
Map<String, LoggerLevel> expectedLevels = Collections.singletonMap( Map<String, LoggerLevel> expectedLevels = Map.of(
"b", "b",
new LoggerLevel(Level.INFO.toString(), null) new LoggerLevel(Level.INFO.toString(), null)
); );
@ -137,7 +135,7 @@ public class MockLoggersTest {
TestLoggers loggers = new TestLoggers(root, x, y, z, w); TestLoggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel("a.b.c.p", Level.WARN.name()); 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.toString(), loggers.level("a.b.c.p").level());
assertEquals(Level.WARN, x.getLevel()); assertEquals(Level.WARN, x.getLevel());
assertEquals(Level.WARN, y.getLevel()); assertEquals(Level.WARN, y.getLevel());
@ -201,7 +199,7 @@ public class MockLoggersTest {
Loggers loggers = new TestLoggers(root, x, y, z, w); Loggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel(rootLoggerName, Level.DEBUG.name()); 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()); 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.ScheduledFuture;
import java.util.concurrent.TimeUnit; 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
@ -94,7 +93,7 @@ public class SourceTaskOffsetCommitterTest {
committer.schedule(taskId, task); committer.schedule(taskId, task);
assertNotNull(taskWrapper.getValue()); assertNotNull(taskWrapper.getValue());
assertEquals(singletonMap(taskId, commitFuture), committers); assertEquals(Map.of(taskId, commitFuture), committers);
} }
@Test @Test

View File

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

View File

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

View File

@ -27,7 +27,6 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -66,13 +65,13 @@ public class WorkerConfigTransformerTest {
@BeforeEach @BeforeEach
public void setup() { public void setup() {
configTransformer = new WorkerConfigTransformer(worker, Collections.singletonMap("test", new TestConfigProvider())); configTransformer = new WorkerConfigTransformer(worker, Map.of("test", new TestConfigProvider()));
} }
@Test @Test
public void testReplaceVariable() { public void testReplaceVariable() {
// Execution // 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 // Assertions
assertEquals(TEST_RESULT, result.get(MY_KEY)); 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); when(herder.restartConnector(eq(1L), eq(MY_CONNECTOR), notNull())).thenReturn(requestId);
// Execution // 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 // Assertions
assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY)); 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); when(herder.restartConnector(eq(10L), eq(MY_CONNECTOR), notNull())).thenReturn(requestId);
// Execution // 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 // Assertions
assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY)); assertEquals(TEST_RESULT_WITH_TTL, result.get(MY_KEY));
verify(herder).restartConnector(eq(1L), eq(MY_CONNECTOR), notNull()); verify(herder).restartConnector(eq(1L), eq(MY_CONNECTOR), notNull());
// Execution // 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 // Assertions
assertEquals(TEST_RESULT_WITH_LONGER_TTL, result.get(MY_KEY)); 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) { public ConfigData get(String path, Set<String> keys) {
if (path.equals(TEST_PATH)) { if (path.equals(TEST_PATH)) {
if (keys.contains(TEST_KEY)) { 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)) { } 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)) { } 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 @Override

View File

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

View File

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

View File

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

View File

@ -31,10 +31,10 @@ import org.mockito.Mockito;
import org.mockito.stubbing.OngoingStubbing; import org.mockito.stubbing.OngoingStubbing;
import java.util.AbstractMap.SimpleEntry; import java.util.AbstractMap.SimpleEntry;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -62,11 +62,11 @@ public class WorkerTestUtils {
connectorConfigs, connectorConfigs,
connectorTargetStates(1, connectorNum, TargetState.STARTED), connectorTargetStates(1, connectorNum, TargetState.STARTED),
taskConfigs(0, connectorNum, connectorNum * taskNum), taskConfigs(0, connectorNum, connectorNum * taskNum),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
appliedConnectorConfigs, appliedConnectorConfigs,
Collections.emptySet(), Set.of(),
Collections.emptySet()); Set.of());
} }
public static Map<String, Integer> connectorTaskCounts(int start, 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 org.junit.jupiter.api.Test;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.List;
import java.util.Collections; 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_V1;
import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2; import static org.apache.kafka.connect.runtime.distributed.IncrementalCooperativeConnectProtocol.CONNECT_PROTOCOL_V2;
@ -99,48 +99,48 @@ public class ConnectProtocolCompatibilityTest {
public void testEagerToEagerAssignment() { public void testEagerToEagerAssignment() {
ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment( ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, 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); ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf); ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed()); assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader()); assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset()); assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors()); assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks()); assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment( ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, 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); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf); ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf);
assertFalse(memberAssignment.failed()); assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader()); assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset()); assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors()); assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks()); assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
} }
@Test @Test
public void testCoopToCoopAssignment() { public void testCoopToCoopAssignment() {
ExtendedAssignment assignment = new ExtendedAssignment( ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0), List.of(connectorId1, connectorId3), List.of(taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf); ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed()); assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader()); assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset()); assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors()); assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks()); assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ExtendedAssignment assignment2 = new ExtendedAssignment( ExtendedAssignment assignment2 = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), List.of(connectorId2), List.of(taskId1x0, taskId3x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.Assignment memberAssignment =
@ -148,15 +148,15 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(memberAssignment.failed()); assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader()); assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset()); assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors()); assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks()); assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
} }
@Test @Test
public void testEagerToCoopAssignment() { public void testEagerToCoopAssignment() {
ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment( ConnectProtocol.Assignment assignment = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, 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); ByteBuffer leaderBuf = ConnectProtocol.serializeAssignment(assignment);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.Assignment leaderAssignment =
@ -164,12 +164,12 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(leaderAssignment.failed()); assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader()); assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset()); assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors()); assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks()); assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment( ConnectProtocol.Assignment assignment2 = new ConnectProtocol.Assignment(
ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, 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); ByteBuffer memberBuf = ConnectProtocol.serializeAssignment(assignment2);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.Assignment memberAssignment =
@ -177,37 +177,37 @@ public class ConnectProtocolCompatibilityTest {
assertFalse(memberAssignment.failed()); assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader()); assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset()); assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors()); assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks()); assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
} }
@Test @Test
public void testCoopToEagerAssignment() { public void testCoopToEagerAssignment() {
ExtendedAssignment assignment = new ExtendedAssignment( ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L, CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "leader", LEADER_URL, 1L,
Arrays.asList(connectorId1, connectorId3), Collections.singletonList(taskId2x0), List.of(connectorId1, connectorId3), List.of(taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); ByteBuffer leaderBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf); ConnectProtocol.Assignment leaderAssignment = ConnectProtocol.deserializeAssignment(leaderBuf);
assertFalse(leaderAssignment.failed()); assertFalse(leaderAssignment.failed());
assertEquals("leader", leaderAssignment.leader()); assertEquals("leader", leaderAssignment.leader());
assertEquals(1, leaderAssignment.offset()); assertEquals(1, leaderAssignment.offset());
assertEquals(Arrays.asList(connectorId1, connectorId3), leaderAssignment.connectors()); assertEquals(List.of(connectorId1, connectorId3), leaderAssignment.connectors());
assertEquals(Collections.singletonList(taskId2x0), leaderAssignment.tasks()); assertEquals(List.of(taskId2x0), leaderAssignment.tasks());
ExtendedAssignment assignment2 = new ExtendedAssignment( ExtendedAssignment assignment2 = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L, CONNECT_PROTOCOL_V1, ConnectProtocol.Assignment.NO_ERROR, "member", LEADER_URL, 1L,
Collections.singletonList(connectorId2), Arrays.asList(taskId1x0, taskId3x0), List.of(connectorId2), List.of(taskId1x0, taskId3x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer memberBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment2, false); ByteBuffer memberBuf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment2, false);
ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf); ConnectProtocol.Assignment memberAssignment = ConnectProtocol.deserializeAssignment(memberBuf);
assertFalse(memberAssignment.failed()); assertFalse(memberAssignment.failed());
assertEquals("member", memberAssignment.leader()); assertEquals("member", memberAssignment.leader());
assertEquals(1, memberAssignment.offset()); assertEquals(1, memberAssignment.offset());
assertEquals(Collections.singletonList(connectorId2), memberAssignment.connectors()); assertEquals(List.of(connectorId2), memberAssignment.connectors());
assertEquals(Arrays.asList(taskId1x0, taskId3x0), memberAssignment.tasks()); assertEquals(List.of(taskId1x0, taskId3x0), memberAssignment.tasks());
} }
private ConnectProtocol.WorkerState emptyWorkerState() { private ConnectProtocol.WorkerState emptyWorkerState() {
@ -221,10 +221,10 @@ public class ConnectProtocolCompatibilityTest {
LEADER, LEADER,
LEADER_URL, LEADER_URL,
CONFIG_OFFSET, CONFIG_OFFSET,
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
0 0
); );
return new ExtendedWorkerState(LEADER_URL, CONFIG_OFFSET, assignment); 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.InvalidParameterException;
import java.security.NoSuchAlgorithmException; import java.security.NoSuchAlgorithmException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -151,9 +148,9 @@ public class DistributedConfigTest {
private void testSupportedAlgorithms(String type, String... expectedAlgorithms) { private void testSupportedAlgorithms(String type, String... expectedAlgorithms) {
Set<String> supportedAlgorithms = DistributedConfig.supportedAlgorithms(type); 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); 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 @Test
@ -214,13 +211,13 @@ public class DistributedConfigTest {
@Test @Test
public void shouldValidateAllVerificationAlgorithms() { public void shouldValidateAllVerificationAlgorithms() {
List<String> algorithms = List<String> algorithms = List.of("HmacSHA1", "HmacSHA256", "HmacMD5", "bad-algorithm");
new ArrayList<>(Arrays.asList("HmacSHA1", "HmacSHA256", "HmacMD5", "bad-algorithm"));
Map<String, String> configs = configs(); Map<String, String> configs = configs();
for (int i = 0; i < algorithms.size(); i++) { for (int i = 0; i < algorithms.size(); i++) {
configs.put(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG, String.join(",", algorithms)); configs.put(DistributedConfig.INTER_WORKER_VERIFICATION_ALGORITHMS_CONFIG, String.join(",", algorithms));
assertThrows(ConfigException.class, () -> new DistributedConfig(configs)); 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.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -662,8 +660,8 @@ public class IncrementalCooperativeAssignorTest {
List<WorkerLoad> expectedAssignment = existingAssignment.stream() List<WorkerLoad> expectedAssignment = existingAssignment.stream()
.map(wl -> new WorkerLoad.Builder(wl.worker()).withCopies(wl.connectors(), wl.tasks()).build()) .map(wl -> new WorkerLoad.Builder(wl.worker()).withCopies(wl.connectors(), wl.tasks()).build())
.collect(Collectors.toList()); .collect(Collectors.toList());
expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9")); expectedAssignment.get(0).connectors().addAll(List.of("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10")); expectedAssignment.get(1).connectors().addAll(List.of("connector7", "connector10"));
expectedAssignment.get(2).connectors().add("connector8"); expectedAssignment.get(2).connectors().add("connector8");
List<String> newConnectors = newConnectors(6, 11); 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()) .map(wl -> new WorkerLoad.Builder(wl.worker()).withCopies(wl.connectors(), wl.tasks()).build())
.collect(Collectors.toList()); .collect(Collectors.toList());
expectedAssignment.get(0).connectors().addAll(Arrays.asList("connector6", "connector9")); expectedAssignment.get(0).connectors().addAll(List.of("connector6", "connector9"));
expectedAssignment.get(1).connectors().addAll(Arrays.asList("connector7", "connector10")); expectedAssignment.get(1).connectors().addAll(List.of("connector7", "connector10"));
expectedAssignment.get(2).connectors().add("connector8"); expectedAssignment.get(2).connectors().add("connector8");
expectedAssignment.get(0).tasks().addAll(Arrays.asList(new ConnectorTaskId("task", 6), new ConnectorTaskId("task", 9))); expectedAssignment.get(0).tasks().addAll(List.of(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(1).tasks().addAll(List.of(new ConnectorTaskId("task", 7), new ConnectorTaskId("task", 10)));
expectedAssignment.get(2).tasks().add(new ConnectorTaskId("task", 8)); expectedAssignment.get(2).tasks().add(new ConnectorTaskId("task", 8));
List<String> newConnectors = newConnectors(6, 11); List<String> newConnectors = newConnectors(6, 11);
@ -734,7 +732,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(), new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -751,7 +749,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -766,7 +764,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(flakyWorker), assertEquals(Set.of(flakyWorker),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -787,7 +785,7 @@ public class IncrementalCooperativeAssignorTest {
.tasks() .tasks()
.containsAll(lostAssignments.tasks()), .containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks"); "Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -814,7 +812,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(), new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -831,7 +829,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -845,7 +843,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -861,7 +859,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors"); "Wrong assignment of lost connectors");
assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()), assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks"); "Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -888,7 +886,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(), new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -908,7 +906,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(newWorker), assertEquals(Set.of(newWorker),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -956,7 +954,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors"); "Wrong assignment of lost connectors");
assertTrue(listOfTasksInLast2Workers.containsAll(lostAssignments.tasks()), assertTrue(listOfTasksInLast2Workers.containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks"); "Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -983,7 +981,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(), new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -1000,7 +998,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -1015,7 +1013,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(), assignor.handleLostAssignments(lostAssignments, new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.singleton(veryFlakyWorker), assertEquals(Set.of(veryFlakyWorker),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance); assertEquals(time.milliseconds() + rebalanceDelay, assignor.scheduledRebalance);
@ -1034,7 +1032,7 @@ public class IncrementalCooperativeAssignorTest {
"Wrong assignment of lost connectors"); "Wrong assignment of lost connectors");
assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()), assertTrue(lostAssignmentsToReassign.build().tasks().containsAll(lostAssignments.tasks()),
"Wrong assignment of lost tasks"); "Wrong assignment of lost tasks");
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -1062,7 +1060,7 @@ public class IncrementalCooperativeAssignorTest {
new ConnectorsAndTasks.Builder(), new ConnectorsAndTasks.Builder(),
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -1080,7 +1078,7 @@ public class IncrementalCooperativeAssignorTest {
assignor.handleLostAssignments(lostAssignments, lostAssignmentsToReassign, assignor.handleLostAssignments(lostAssignments, lostAssignmentsToReassign,
new ArrayList<>(configuredAssignment.values())); new ArrayList<>(configuredAssignment.values()));
assertEquals(Collections.emptySet(), assertEquals(Set.of(),
assignor.candidateWorkersForReassignment, assignor.candidateWorkersForReassignment,
"Wrong set of workers for reassignments"); "Wrong set of workers for reassignments");
assertEquals(0, assignor.scheduledRebalance); assertEquals(0, assignor.scheduledRebalance);
@ -1221,10 +1219,10 @@ public class IncrementalCooperativeAssignorTest {
leader, leader,
"followMe:618", "followMe:618",
CONFIG_OFFSET, CONFIG_OFFSET,
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
0 0
); );
ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment); ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment);
@ -1262,10 +1260,10 @@ public class IncrementalCooperativeAssignorTest {
leader, leader,
"followMe:618", "followMe:618",
CONFIG_OFFSET, CONFIG_OFFSET,
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
0 0
); );
ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment); ExtendedWorkerState leaderState = new ExtendedWorkerState("followMe:618", CONFIG_OFFSET, leaderAssignment);
@ -1328,7 +1326,7 @@ public class IncrementalCooperativeAssignorTest {
private void addNewEmptyWorkers(String... workers) { private void addNewEmptyWorkers(String... workers) {
for (String worker : 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() { private ClusterConfigState configState() {
Map<String, Integer> taskCounts = new HashMap<>(connectors); 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<String, TargetState> targetStates = transformValues(taskCounts, c -> TargetState.STARTED);
Map<ConnectorTaskId, Map<String, String>> taskConfigs = taskCounts.entrySet().stream() Map<ConnectorTaskId, Map<String, String>> taskConfigs = taskCounts.entrySet().stream()
.flatMap(e -> IntStream.range(0, e.getValue()).mapToObj(i -> new ConnectorTaskId(e.getKey(), i))) .flatMap(e -> IntStream.range(0, e.getValue()).mapToObj(i -> new ConnectorTaskId(e.getKey(), i)))
.collect(Collectors.toMap( .collect(Collectors.toMap(
Function.identity(), Function.identity(),
connectorTaskId -> Collections.emptyMap() connectorTaskId -> Map.of()
)); ));
Map<String, AppliedConnectorConfig> appliedConnectorConfigs = connectorConfigs.entrySet().stream() Map<String, AppliedConnectorConfig> appliedConnectorConfigs = connectorConfigs.entrySet().stream()
.collect(Collectors.toMap( .collect(Collectors.toMap(
@ -1412,11 +1410,11 @@ public class IncrementalCooperativeAssignorTest {
connectorConfigs, connectorConfigs,
targetStates, targetStates,
taskConfigs, taskConfigs,
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
appliedConnectorConfigs, appliedConnectorConfigs,
Collections.emptySet(), Set.of(),
Collections.emptySet()); Set.of());
} }
private void applyAssignments() { private void applyAssignments() {
@ -1440,16 +1438,16 @@ public class IncrementalCooperativeAssignorTest {
} }
private void assertEmptyAssignment() { private void assertEmptyAssignment() {
assertEquals(Collections.emptyList(), assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyAssignedConnectors().values()), ConnectUtils.combineCollections(returnedAssignments.newlyAssignedConnectors().values()),
"No connectors should have been newly assigned during this round"); "No connectors should have been newly assigned during this round");
assertEquals(Collections.emptyList(), assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyAssignedTasks().values()), ConnectUtils.combineCollections(returnedAssignments.newlyAssignedTasks().values()),
"No tasks should have been newly assigned during this round"); "No tasks should have been newly assigned during this round");
assertEquals(Collections.emptyList(), assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyRevokedConnectors().values()), ConnectUtils.combineCollections(returnedAssignments.newlyRevokedConnectors().values()),
"No connectors should have been revoked during this round"); "No connectors should have been revoked during this round");
assertEquals(Collections.emptyList(), assertEquals(List.of(),
ConnectUtils.combineCollections(returnedAssignments.newlyRevokedTasks().values()), ConnectUtils.combineCollections(returnedAssignments.newlyRevokedTasks().values()),
"No tasks should have been revoked during this round"); "No tasks should have been revoked during this round");
} }
@ -1500,14 +1498,14 @@ public class IncrementalCooperativeAssignorTest {
private void assertNoRevocations() { private void assertNoRevocations() {
returnedAssignments.newlyRevokedConnectors().forEach((worker, revocations) -> returnedAssignments.newlyRevokedConnectors().forEach((worker, revocations) ->
assertEquals( assertEquals(
Collections.emptySet(), Set.of(),
new HashSet<>(revocations), new HashSet<>(revocations),
"Expected no revocations to take place during this round, but connector revocations were issued for worker " + worker "Expected no revocations to take place during this round, but connector revocations were issued for worker " + worker
) )
); );
returnedAssignments.newlyRevokedTasks().forEach((worker, revocations) -> returnedAssignments.newlyRevokedTasks().forEach((worker, revocations) ->
assertEquals( assertEquals(
Collections.emptySet(), Set.of(),
new HashSet<>(revocations), new HashSet<>(revocations),
"Expected no revocations to take place during this round, but task revocations were issued for worker " + worker "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); existingConnectors.retainAll(newConnectors);
assertEquals(Collections.emptyList(), assertEquals(List.of(),
existingConnectors, existingConnectors,
"Found connectors in new assignment that already exist in current assignment"); "Found connectors in new assignment that already exist in current assignment");
existingTasks.retainAll(newTasks); existingTasks.retainAll(newTasks);
assertEquals(Collections.emptyList(), assertEquals(List.of(),
existingConnectors, existingConnectors,
"Found tasks in new assignment that already exist in current assignment"); "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.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -126,7 +124,7 @@ public class WorkerCoordinatorIncrementalTest {
this.time = new MockTime(); this.time = new MockTime();
this.metadata = new Metadata(0, 0, Long.MAX_VALUE, loggerFactory, new ClusterResourceListeners()); this.metadata = new Metadata(0, 0, Long.MAX_VALUE, loggerFactory, new ClusterResourceListeners());
this.client = new MockClient(time, metadata); 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.node = metadata.fetch().nodes().get(0);
this.consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time, this.consumerClient = new ConsumerNetworkClient(loggerFactory, client, metadata, time,
retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs); retryBackoffMs, requestTimeoutMs, heartbeatIntervalMs);
@ -210,8 +208,8 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment assignment = new ExtendedAssignment( ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(), CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(),
Collections.singletonList(connectorId1), Arrays.asList(taskId1x0, taskId2x0), List.of(connectorId1), List.of(taskId1x0, taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
// Using onJoinComplete to register the protocol selection decided by the broker // 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 // 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())); .deserializeMetadata(ByteBuffer.wrap(selectedMetadata.metadata()));
assertEquals(offset, state.offset()); assertEquals(offset, state.offset());
assertNotEquals(ExtendedAssignment.empty(), state.assignment()); assertNotEquals(ExtendedAssignment.empty(), state.assignment());
assertEquals(Collections.singletonList(connectorId1), state.assignment().connectors()); assertEquals(List.of(connectorId1), state.assignment().connectors());
assertEquals(Arrays.asList(taskId1x0, taskId2x0), state.assignment().tasks()); assertEquals(List.of(taskId1x0, taskId2x0), state.assignment().tasks());
verify(configStorage, times(1)).snapshot(); verify(configStorage, times(1)).snapshot();
} }
@ -243,8 +241,8 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment assignment = new ExtendedAssignment( ExtendedAssignment assignment = new ExtendedAssignment(
CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(), CONNECT_PROTOCOL_V1, ExtendedAssignment.NO_ERROR, leaderId, leaderUrl, configState1.offset(),
Collections.singletonList(connectorId1), Arrays.asList(taskId1x0, taskId2x0), List.of(connectorId1), List.of(taskId1x0, taskId2x0),
Collections.emptyList(), Collections.emptyList(), 0); List.of(), List.of(), 0);
ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false); ByteBuffer buf = IncrementalCooperativeConnectProtocol.serializeAssignment(assignment, false);
// Using onJoinComplete to register the protocol selection decided by the broker // 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 // 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); ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 4, List.of(connectorId1), 4,
Collections.emptyList(), 0, List.of(), 0,
leaderAssignment); leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId); ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 4, List.of(connectorId2), 4,
Collections.emptyList(), 0, List.of(), 0,
memberAssignment); memberAssignment);
coordinator.metadata(); coordinator.metadata();
@ -306,20 +304,20 @@ public class WorkerCoordinatorIncrementalTest {
//Equally distributing tasks across member //Equally distributing tasks across member
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 1, List.of(), 1,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 1, List.of(), 1,
memberAssignment); memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId); ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
anotherMemberAssignment); anotherMemberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot(); verify(configStorage, times(configStorageCalls)).snapshot();
@ -344,20 +342,20 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId); ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 3, List.of(connectorId1), 3,
Collections.emptyList(), 0, List.of(), 0,
leaderAssignment); leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId); ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 3, List.of(connectorId2), 3,
Collections.emptyList(), 0, List.of(), 0,
memberAssignment); memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId); ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 2, List.of(), 2,
Collections.emptyList(), 0, List.of(), 0,
anotherMemberAssignment); anotherMemberAssignment);
// Second rebalance detects a worker is missing // Second rebalance detects a worker is missing
@ -373,15 +371,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
memberAssignment); memberAssignment);
@ -393,15 +391,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
memberAssignment); memberAssignment);
@ -412,14 +410,14 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 1, List.of(), 1,
Collections.emptyList(), 0, List.of(), 0,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 1, List.of(), 1,
Collections.emptyList(), 0, List.of(), 0,
memberAssignment); memberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot(); verify(configStorage, times(configStorageCalls)).snapshot();
@ -444,20 +442,20 @@ public class WorkerCoordinatorIncrementalTest {
ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId); ExtendedAssignment leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId1), 3, List.of(connectorId1), 3,
Collections.emptyList(), 0, List.of(), 0,
leaderAssignment); leaderAssignment);
ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId); ExtendedAssignment memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.singletonList(connectorId2), 3, List.of(connectorId2), 3,
Collections.emptyList(), 0, List.of(), 0,
memberAssignment); memberAssignment);
ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId); ExtendedAssignment anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 2, List.of(), 2,
Collections.emptyList(), 0, List.of(), 0,
anotherMemberAssignment); anotherMemberAssignment);
// Second rebalance detects a worker is missing // Second rebalance detects a worker is missing
@ -472,15 +470,15 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
memberAssignment); memberAssignment);
@ -494,22 +492,22 @@ public class WorkerCoordinatorIncrementalTest {
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
memberAssignment); memberAssignment);
anotherMemberAssignment = deserializeAssignment(result, anotherMemberId); anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
rebalanceDelay, rebalanceDelay,
anotherMemberAssignment); anotherMemberAssignment);
@ -520,20 +518,20 @@ public class WorkerCoordinatorIncrementalTest {
// A rebalance after the delay expires re-assigns the lost tasks to the returning member // A rebalance after the delay expires re-assigns the lost tasks to the returning member
leaderAssignment = deserializeAssignment(result, leaderId); leaderAssignment = deserializeAssignment(result, leaderId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
leaderAssignment); leaderAssignment);
memberAssignment = deserializeAssignment(result, memberId); memberAssignment = deserializeAssignment(result, memberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 0, List.of(), 0,
Collections.emptyList(), 0, List.of(), 0,
memberAssignment); memberAssignment);
anotherMemberAssignment = deserializeAssignment(result, anotherMemberId); anotherMemberAssignment = deserializeAssignment(result, anotherMemberId);
assertAssignment(leaderId, offset, assertAssignment(leaderId, offset,
Collections.emptyList(), 2, List.of(), 2,
Collections.emptyList(), 0, List.of(), 0,
anotherMemberAssignment); anotherMemberAssignment);
verify(configStorage, times(configStorageCalls)).snapshot(); verify(configStorage, times(configStorageCalls)).snapshot();
@ -543,8 +541,8 @@ public class WorkerCoordinatorIncrementalTest {
public ExtendedAssignment assignment = null; public ExtendedAssignment assignment = null;
public String revokedLeader; public String revokedLeader;
public Collection<String> revokedConnectors = Collections.emptyList(); public Collection<String> revokedConnectors = List.of();
public Collection<ConnectorTaskId> revokedTasks = Collections.emptyList(); public Collection<ConnectorTaskId> revokedTasks = List.of();
public int revokedCount = 0; public int revokedCount = 0;
public int assignedCount = 0; public int assignedCount = 0;

View File

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

View File

@ -46,8 +46,6 @@ import java.util.Map;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Future; 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_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;
import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE; import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE;
@ -105,13 +103,13 @@ public class ErrorReporterTest {
@Test @Test
public void initializeDLQWithNullMetrics() { 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 @Test
public void testDLQConfigWithEmptyTopicName() { public void testDLQConfigWithEmptyTopicName() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter(
producer, config(emptyMap()), TASK_ID, errorHandlingMetrics); producer, config(Map.of()), TASK_ID, errorHandlingMetrics);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -124,7 +122,7 @@ public class ErrorReporterTest {
@Test @Test
public void testDLQConfigWithValidTopicName() { public void testDLQConfigWithValidTopicName() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( 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(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -138,7 +136,7 @@ public class ErrorReporterTest {
@Test @Test
public void testReportDLQTwice() { public void testReportDLQTwice() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( 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(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -153,7 +151,7 @@ public class ErrorReporterTest {
@Test @Test
public void testCloseDLQ() { public void testCloseDLQ() {
DeadLetterQueueReporter deadLetterQueueReporter = new DeadLetterQueueReporter( 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(); deadLetterQueueReporter.close();
verify(producer).close(); verify(producer).close();
@ -161,7 +159,7 @@ public class ErrorReporterTest {
@Test @Test
public void testLogOnDisabledLogReporter() { 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(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
context.error(new RuntimeException()); context.error(new RuntimeException());
@ -173,7 +171,7 @@ public class ErrorReporterTest {
@Test @Test
public void testLogOnEnabledLogReporter() { 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(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
context.error(new RuntimeException()); context.error(new RuntimeException());
@ -185,7 +183,7 @@ public class ErrorReporterTest {
@Test @Test
public void testLogMessageWithNoRecords() { 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(); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = processingContext();
@ -231,10 +229,10 @@ public class ErrorReporterTest {
@Test @Test
public void testSetDLQConfigs() { 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()); 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()); assertEquals(7, configuration.dlqTopicReplicationFactor());
} }

View File

@ -44,8 +44,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import org.mockito.stubbing.OngoingStubbing; import org.mockito.stubbing.OngoingStubbing;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -54,11 +52,8 @@ import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future; import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream; 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.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_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_MAX_DELAY_DEFAULT; import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_MAX_DELAY_DEFAULT;
@ -232,22 +227,22 @@ public class RetryWithToleranceOperatorTest {
@Test @Test
public void testExecAndHandleRetriableErrorOnce() throws Exception { 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 @Test
public void testExecAndHandleRetriableErrorThrice() throws Exception { 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 @Test
public void testExecAndHandleRetriableErrorWithInfiniteRetries() throws Exception { 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 @Test
public void testExecAndHandleRetriableErrorWithMaxRetriesExceeded() throws Exception { 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 { public void execAndHandleRetriableError(long errorRetryTimeout, int numRetriableExceptionsThrown, List<Long> expectedWaits, Exception e, boolean successExpected) throws Exception {
@ -396,7 +391,7 @@ public class RetryWithToleranceOperatorTest {
@Test @Test
public void testDefaultConfigs() { public void testDefaultConfigs() {
ConnectorConfig configuration = config(emptyMap()); ConnectorConfig configuration = config(Map.of());
assertEquals(ERRORS_RETRY_TIMEOUT_DEFAULT, configuration.errorRetryTimeout()); assertEquals(ERRORS_RETRY_TIMEOUT_DEFAULT, configuration.errorRetryTimeout());
assertEquals(ERRORS_RETRY_MAX_DELAY_DEFAULT, configuration.errorMaxDelayInMillis()); assertEquals(ERRORS_RETRY_MAX_DELAY_DEFAULT, configuration.errorMaxDelayInMillis());
assertEquals(ERRORS_TOLERANCE_DEFAULT, configuration.errorToleranceType()); assertEquals(ERRORS_TOLERANCE_DEFAULT, configuration.errorToleranceType());
@ -413,13 +408,13 @@ public class RetryWithToleranceOperatorTest {
@Test @Test
public void testSetConfigs() { public void testSetConfigs() {
ConnectorConfig configuration; ConnectorConfig configuration;
configuration = config(singletonMap(ERRORS_RETRY_TIMEOUT_CONFIG, "100")); configuration = config(Map.of(ERRORS_RETRY_TIMEOUT_CONFIG, "100"));
assertEquals(100, configuration.errorRetryTimeout()); 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()); assertEquals(100, configuration.errorMaxDelayInMillis());
configuration = config(singletonMap(ERRORS_TOLERANCE_CONFIG, "none")); configuration = config(Map.of(ERRORS_TOLERANCE_CONFIG, "none"));
assertEquals(ToleranceType.NONE, configuration.errorToleranceType()); 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); 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); 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<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); retryWithToleranceOperator.reporters(reporters);
ProcessingContext<ConsumerRecord<byte[], byte[]>> context = new ProcessingContext<>(consumerRecord); ProcessingContext<ConsumerRecord<byte[], byte[]>> context = new ProcessingContext<>(consumerRecord);
Future<Void> result = retryWithToleranceOperator.report(context); Future<Void> result = retryWithToleranceOperator.report(context);
@ -458,7 +453,7 @@ public class RetryWithToleranceOperatorTest {
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = allOperator(); 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. // Even though the reporters throw exceptions, they should both still be closed.
@ -475,7 +470,7 @@ public class RetryWithToleranceOperatorTest {
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator = allOperator(); 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. // Even though the reporters throw exceptions, they should both still be closed.
doThrow(new RuntimeException()).when(reporterA).close(); doThrow(new RuntimeException()).when(reporterA).close();

View File

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

View File

@ -29,9 +29,8 @@ import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
@ -55,7 +54,7 @@ public class ConnectClusterStateImplTest {
@BeforeEach @BeforeEach
public void setUp() { public void setUp() {
expectedConnectors = Arrays.asList("sink1", "source1", "source2"); expectedConnectors = List.of("sink1", "source1", "source2");
connectClusterState = new ConnectClusterStateImpl( connectClusterState = new ConnectClusterStateImpl(
herderRequestTimeoutMs, herderRequestTimeoutMs,
new ConnectClusterDetailsImpl(KAFKA_CLUSTER_ID), new ConnectClusterDetailsImpl(KAFKA_CLUSTER_ID),
@ -78,7 +77,7 @@ public class ConnectClusterStateImplTest {
@Test @Test
public void connectorConfig() { public void connectorConfig() {
final String connName = "sink6"; final String connName = "sink6";
final Map<String, String> expectedConfig = Collections.singletonMap("key", "value"); final Map<String, String> expectedConfig = Map.of("key", "value");
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
ArgumentCaptor<Callback<Map<String, String>>> callback = ArgumentCaptor.forClass(Callback.class); 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.MalformedURLException;
import java.net.URL; import java.net.URL;
import java.util.Collections;
import java.util.SortedSet; import java.util.SortedSet;
import java.util.TreeSet; import java.util.TreeSet;
@ -73,15 +72,15 @@ public class DelegatingClassLoaderTest {
assertTrue(PluginUtils.shouldLoadInIsolation(pluginDesc.className())); assertTrue(PluginUtils.shouldLoadInIsolation(pluginDesc.className()));
sinkConnectors.add(pluginDesc); sinkConnectors.add(pluginDesc);
scanResult = new PluginScanResult( scanResult = new PluginScanResult(
sinkConnectors, sinkConnectors,
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet(), new TreeSet<>(),
Collections.emptySortedSet() new TreeSet<>()
); );
} }

View File

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

View File

@ -24,7 +24,6 @@ import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -51,12 +50,12 @@ public class PluginRecommenderTest {
@Test @Test
public void testConnectorVersionRecommenders() { public void testConnectorVersionRecommenders() {
PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS); PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
for (String connectorClass : Arrays.asList( for (String connectorClass : List.of(
VersionedPluginBuilder.VersionedTestPlugin.SINK_CONNECTOR.className(), VersionedPluginBuilder.VersionedTestPlugin.SINK_CONNECTOR.className(),
VersionedPluginBuilder.VersionedTestPlugin.SOURCE_CONNECTOR.className()) VersionedPluginBuilder.VersionedTestPlugin.SOURCE_CONNECTOR.className())
) { ) {
Set<String> versions = recommender.connectorPluginVersionRecommender().validValues( 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()); ).stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(connectorClass); Set<String> allVersions = allVersionsOf(connectorClass);
Assertions.assertEquals(allVersions.size(), versions.size()); Assertions.assertEquals(allVersions.size(), versions.size());
@ -73,7 +72,7 @@ public class PluginRecommenderTest {
config.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, converterClass); config.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, converterClass);
config.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, converterClass); config.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, converterClass);
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.CONVERTER.className()); 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()); Set<String> versions = r.validValues(null, config).stream().map(Object::toString).collect(Collectors.toSet());
Assertions.assertEquals(allVersions.size(), versions.size()); Assertions.assertEquals(allVersions.size(), versions.size());
allVersions.forEach(v -> Assertions.assertTrue(versions.contains(v), "Missing version " + v + " for converter")); 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); PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
Class transformationClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className()); Class transformationClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className());
Set<String> versions = recommender.transformationPluginRecommender("transforms.t1.type") 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()); .stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className()); Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.TRANSFORMATION.className());
Assertions.assertEquals(allVersions.size(), versions.size()); Assertions.assertEquals(allVersions.size(), versions.size());
@ -112,7 +111,7 @@ public class PluginRecommenderTest {
PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS); PluginsRecommenders recommender = new PluginsRecommenders(MULTI_VERSION_PLUGINS);
Class predicateClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className()); Class predicateClass = MULTI_VERSION_PLUGINS.pluginClass(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className());
Set<String> versions = recommender.predicatePluginRecommender("predicates.p1.type") 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()); .stream().map(Object::toString).collect(Collectors.toSet());
Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className()); Set<String> allVersions = allVersionsOf(VersionedPluginBuilder.VersionedTestPlugin.PREDICATE.className());
Assertions.assertEquals(allVersions.size(), versions.size()); 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.io.File;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
@ -50,7 +49,7 @@ public class PluginScannerTest {
@ParameterizedTest @ParameterizedTest
@MethodSource("parameters") @MethodSource("parameters")
public void testScanningEmptyPluginPath(PluginScanner scanner) { public void testScanningEmptyPluginPath(PluginScanner scanner) {
PluginScanResult result = scan(scanner, Collections.emptySet()); PluginScanResult result = scan(scanner, Set.of());
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
} }
@ -69,7 +68,7 @@ public class PluginScannerTest {
public void testScanningInvalidUberJar(PluginScanner scanner) throws Exception { public void testScanningInvalidUberJar(PluginScanner scanner) throws Exception {
File newFile = new File(pluginDir, "invalid.jar"); File newFile = new File(pluginDir, "invalid.jar");
newFile.createNewFile(); newFile.createNewFile();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath())); PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
} }
@ -81,14 +80,14 @@ public class PluginScannerTest {
newFile = new File(newFile, "invalid.jar"); newFile = new File(newFile, "invalid.jar");
newFile.createNewFile(); newFile.createNewFile();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath())); PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
} }
@ParameterizedTest @ParameterizedTest
@MethodSource("parameters") @MethodSource("parameters")
public void testScanningNoPlugins(PluginScanner scanner) { public void testScanningNoPlugins(PluginScanner scanner) {
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath())); PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
} }
@ -98,7 +97,7 @@ public class PluginScannerTest {
File newFile = new File(pluginDir, "my-plugin"); File newFile = new File(pluginDir, "my-plugin");
newFile.mkdir(); newFile.mkdir();
PluginScanResult result = scan(scanner, Collections.singleton(pluginDir.toPath())); PluginScanResult result = scan(scanner, Set.of(pluginDir.toPath()));
assertTrue(result.isEmpty()); assertTrue(result.isEmpty());
} }
@ -116,7 +115,7 @@ public class PluginScannerTest {
Files.copy(source, pluginPath.resolve(source.getFileName())); 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<>(); Set<String> classes = new HashSet<>();
result.forEach(pluginDesc -> classes.add(pluginDesc.className())); result.forEach(pluginDesc -> classes.add(pluginDesc.className()));
Set<String> expectedClasses = new HashSet<>(TestPlugins.pluginClasses()); Set<String> expectedClasses = new HashSet<>(TestPlugins.pluginClasses());

View File

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

View File

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

View File

@ -17,7 +17,6 @@
package org.apache.kafka.connect.runtime.isolation; package org.apache.kafka.connect.runtime.isolation;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -42,7 +41,7 @@ public interface SamplingTestPlugin {
* @return All known instances of this class, including this instance. * @return All known instances of this class, including this instance.
*/ */
default List<SamplingTestPlugin> allInstances() { 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 * This should only return direct children, and not reference this instance directly
*/ */
default Map<String, SamplingTestPlugin> otherSamples() { 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.lang.management.ThreadInfo;
import java.net.URL; import java.net.URL;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.BrokenBarrierException;
@ -71,7 +70,7 @@ public class SynchronizationTest {
@BeforeEach @BeforeEach
public void setup(TestInfo testInfo) { public void setup(TestInfo testInfo) {
Map<String, String> pluginProps = Collections.singletonMap( Map<String, String> pluginProps = Map.of(
WorkerConfig.PLUGIN_PATH_CONFIG, WorkerConfig.PLUGIN_PATH_CONFIG,
TestPlugins.pluginPathJoined() TestPlugins.pluginPathJoined()
); );
@ -241,7 +240,7 @@ public class SynchronizationTest {
// 4. Load the isolated plugin class and return // 4. Load the isolated plugin class and return
new AbstractConfig( new AbstractConfig(
new ConfigDef().define("a.class", Type.CLASS, Importance.HIGH, ""), 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 // 3. Enter the DelegatingClassLoader
// 4. Load the non-isolated class and return // 4. Load the non-isolated class and return
new AbstractConfig(new ConfigDef().define("a.class", Type.CLASS, Importance.HIGH, ""), 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)) { if (pluginJars.containsKey(testPackage)) {
log.debug("Skipping recompilation of {}", testPackage.resourceDir()); 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) { } catch (Throwable e) {
log.error("Could not set up plugin test jars", e); log.error("Could not set up plugin test jars", e);
@ -372,7 +372,7 @@ public class TestPlugins {
.filter(Objects::nonNull) .filter(Objects::nonNull)
.map(TestPlugin::className) .map(TestPlugin::className)
.distinct() .distinct()
.collect(Collectors.toList()); .toList();
} }
public static Function<ClassLoader, LoaderSwap> noOpLoaderSwap() { public static Function<ClassLoader, LoaderSwap> noOpLoaderSwap() {
@ -458,11 +458,11 @@ public class TestPlugins {
.map(Path::toFile) .map(Path::toFile)
.filter(file -> file.getName().endsWith(".java")) .filter(file -> file.getName().endsWith(".java"))
.map(file -> replacements.isEmpty() ? file : copyAndReplace(file, replacements)) .map(file -> replacements.isEmpty() ? file : copyAndReplace(file, replacements))
.collect(Collectors.toList()); .toList();
} }
StringWriter writer = new StringWriter(); StringWriter writer = new StringWriter();
List<String> options = Arrays.asList( List<String> options = List.of(
"-d", binDir.toString() // Write class output to a different directory. "-d", binDir.toString() // Write class output to a different directory.
); );
try (StandardJavaFileManager fileManager = compiler.getStandardFileManager(null, null, null)) { 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.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map;
public class VersionedPluginBuilder { public class VersionedPluginBuilder {
@ -99,7 +99,7 @@ public class VersionedPluginBuilder {
Path subDir = Files.createDirectory(pluginDirPath.resolve("lib")); Path subDir = Files.createDirectory(pluginDirPath.resolve("lib"));
subDir.toFile().deleteOnExit(); subDir.toFile().deleteOnExit();
for (BuildInfo buildInfo : pluginBuilds) { 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(); Path targetJar = subDir.resolve(jarFile.getFileName()).toAbsolutePath();
buildInfo.setLocation(targetJar.toString()); buildInfo.setLocation(targetJar.toString());
targetJar.toFile().deleteOnExit(); targetJar.toFile().deleteOnExit();

View File

@ -60,9 +60,7 @@ import java.io.IOException;
import java.net.URI; import java.net.URI;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashMap; import java.util.LinkedHashMap;
import java.util.List; import java.util.List;
@ -213,7 +211,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId(); doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins(); doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions(); expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors(); doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap); server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer(); server.initializeServer();
@ -256,7 +254,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId(); doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins(); doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions(); expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors(); doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap); server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer(); server.initializeServer();
@ -278,8 +276,8 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId(); doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins(); doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions(); expectEmptyRestExtensions();
doReturn(Collections.emptyList()).when(herder).setWorkerLoggerLevel(logger, loggingLevel); doReturn(List.of()).when(herder).setWorkerLoggerLevel(logger, loggingLevel);
doReturn(Collections.singletonMap(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels(); doReturn(Map.of(logger, new LoggerLevel(loggingLevel, lastModified))).when(herder).allLoggerLevels();
server = new ConnectRestServer(null, restClient, configMap); server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer(); server.initializeServer();
@ -296,7 +294,7 @@ public class ConnectRestServerTest {
Map<String, Object> expectedLogger = new HashMap<>(); Map<String, Object> expectedLogger = new HashMap<>();
expectedLogger.put("level", loggingLevel); expectedLogger.put("level", loggingLevel);
expectedLogger.put("last_modified", lastModified); 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<>() { }); Map<String, Map<String, Object>> actualLoggers = mapper.readValue(responseStr, new TypeReference<>() { });
assertEquals(expectedLoggers, actualLoggers); assertEquals(expectedLoggers, actualLoggers);
} }
@ -438,7 +436,7 @@ public class ConnectRestServerTest {
doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId(); doReturn(KAFKA_CLUSTER_ID).when(herder).kafkaClusterId();
doReturn(plugins).when(herder).plugins(); doReturn(plugins).when(herder).plugins();
expectEmptyRestExtensions(); expectEmptyRestExtensions();
doReturn(Arrays.asList("a", "b")).when(herder).connectors(); doReturn(List.of("a", "b")).when(herder).connectors();
server = new ConnectRestServer(null, restClient, configMap); server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer(); server.initializeServer();
@ -481,8 +479,8 @@ public class ConnectRestServerTest {
} }
private void expectEmptyRestExtensions() { private void expectEmptyRestExtensions() {
doReturn(Collections.emptyList()).when(plugins).newPlugins( doReturn(List.of()).when(plugins).newPlugins(
eq(Collections.emptyList()), eq(List.of()),
any(AbstractConfig.class), any(AbstractConfig.class),
eq(ConnectRestExtension.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 org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -35,7 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class RestServerConfigTest { 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 \t Cache-Control: no-cache, no-store, must-revalidate",
"add \r X-XSS-Protection: 1; mode=block", "add \r X-XSS-Protection: 1; mode=block",
"\n add Strict-Transport-Security: max-age=31536000; includeSubDomains", "\n add Strict-Transport-Security: max-age=31536000; includeSubDomains",
@ -48,7 +46,7 @@ public class RestServerConfigTest {
"adDdate \n Last-Modified: \t 0" "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", "set \t",
"badaction \t X-Frame-Options:DENY", "badaction \t X-Frame-Options:DENY",
"set add X-XSS-Protection:1", "set add X-XSS-Protection:1",
@ -70,11 +68,11 @@ public class RestServerConfigTest {
props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999"); props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999");
config = RestServerConfig.forPublic(null, props); 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"); props.put(RestServerConfig.LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
config = RestServerConfig.forPublic(null, props); 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 @Test
@ -113,7 +111,7 @@ public class RestServerConfigTest {
props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812"); props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999, https://a.b:7812");
config = RestServerConfig.forPublic(null, props); 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); 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 org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertEquals;
@ -43,7 +43,7 @@ public class ConnectorOffsetsTest {
offset2.put("offset", new byte[]{0x00, 0x1A}); offset2.put("offset", new byte[]{0x00, 0x1A});
ConnectorOffset connectorOffset2 = new ConnectorOffset(partition2, offset2); 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(); Map<Map<String, ?>, Map<String, ?>> connectorOffsetsMap = connectorOffsets.toMap();
assertEquals(2, connectorOffsetsMap.size()); assertEquals(2, connectorOffsetsMap.size());
assertEquals(offset1, connectorOffsetsMap.get(partition1)); 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 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.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertNull;
@ -33,7 +33,7 @@ public class CreateConnectorRequestTest {
assertEquals(TargetState.PAUSED, CreateConnectorRequest.InitialState.PAUSED.toTargetState()); assertEquals(TargetState.PAUSED, CreateConnectorRequest.InitialState.PAUSED.toTargetState());
assertEquals(TargetState.STOPPED, CreateConnectorRequest.InitialState.STOPPED.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()); assertNull(createConnectorRequest.initialTargetState());
} }

View File

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

View File

@ -55,9 +55,7 @@ import org.mockito.stubbing.Stubber;
import java.net.URI; import java.net.URI;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; 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(ConnectorConfig.NAME_CONFIG, "");
CONNECTOR_CONFIG_WITH_EMPTY_NAME.put("sample_config", "test_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, 0),
new ConnectorTaskId(CONNECTOR_NAME, 1) new ConnectorTaskId(CONNECTOR_NAME, 1)
); );
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>(); private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
static { static {
TASK_CONFIGS.add(Collections.singletonMap("config", "value")); TASK_CONFIGS.add(Map.of("config", "value"));
TASK_CONFIGS.add(Collections.singletonMap("config", "other_value")); TASK_CONFIGS.add(Map.of("config", "other_value"));
} }
private static final List<TaskInfo> TASK_INFOS = new ArrayList<>(); private static final List<TaskInfo> TASK_INFOS = new ArrayList<>();
static { static {
@ -158,7 +156,7 @@ public class ConnectorsResourceTest {
} }
private static final Set<String> CONNECTOR_ACTIVE_TOPICS = new HashSet<>( 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( private static final RestRequestTimeout REQUEST_TIMEOUT = RestRequestTimeout.constant(
DEFAULT_REST_REQUEST_TIMEOUT_MS, DEFAULT_REST_REQUEST_TIMEOUT_MS,
@ -196,7 +194,7 @@ public class ConnectorsResourceTest {
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>(); MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>();
queryParams.putSingle("forward", "true"); queryParams.putSingle("forward", "true");
when(forward.getQueryParameters()).thenReturn(queryParams); 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(); Collection<String> connectors = (Collection<String>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets // Ordering isn't guaranteed, compare sets
@ -205,7 +203,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testExpandConnectorsStatus() { 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 connector = mock(ConnectorStateInfo.class);
ConnectorStateInfo connector2 = mock(ConnectorStateInfo.class); ConnectorStateInfo connector2 = mock(ConnectorStateInfo.class);
when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2); when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2);
@ -225,7 +223,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testExpandConnectorsInfo() { 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 connector = mock(ConnectorInfo.class);
ConnectorInfo connector2 = mock(ConnectorInfo.class); ConnectorInfo connector2 = mock(ConnectorInfo.class);
when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connector2); when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connector2);
@ -245,7 +243,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testFullExpandConnectors() { 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 connectorInfo = mock(ConnectorInfo.class);
ConnectorInfo connectorInfo2 = mock(ConnectorInfo.class); ConnectorInfo connectorInfo2 = mock(ConnectorInfo.class);
when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connectorInfo2); when(herder.connectorInfo(CONNECTOR2_NAME)).thenReturn(connectorInfo2);
@ -257,7 +255,7 @@ public class ConnectorsResourceTest {
forward = mock(UriInfo.class); forward = mock(UriInfo.class);
MultivaluedMap<String, String> queryParams = new MultivaluedHashMap<>(); 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); when(forward.getQueryParameters()).thenReturn(queryParams);
Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity(); 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 @Test
public void testExpandConnectorsWithConnectorNotFound() { 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); ConnectorStateInfo connector2 = mock(ConnectorStateInfo.class);
when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2); when(herder.connectorStatus(CONNECTOR2_NAME)).thenReturn(connector2);
doThrow(mock(NotFoundException.class)).when(herder).connectorStatus(CONNECTOR_NAME); 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(); Map<String, Map<String, Object>> expanded = (Map<String, Map<String, Object>>) connectorsResource.listConnectors(forward, NULL_HEADERS).getEntity();
// Ordering isn't guaranteed, compare sets // 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")); assertEquals(connector2, expanded.get(CONNECTOR2_NAME).get("status"));
} }
@ -291,7 +289,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnector() throws Throwable { public void testCreateConnector() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -304,7 +302,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithPausedInitialState() throws Throwable { public void testCreateConnectorWithPausedInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -317,7 +315,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithStoppedInitialState() throws Throwable { public void testCreateConnectorWithStoppedInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -330,7 +328,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithRunningInitialState() throws Throwable { public void testCreateConnectorWithRunningInitialState() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG,
@ -343,7 +341,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorNotLeader() throws Throwable { public void testCreateConnectorNotLeader() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackNotLeaderException(cb).when(herder) expectAndCallbackNotLeaderException(cb).when(herder)
@ -357,7 +355,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithHeaders() throws Throwable { public void testCreateConnectorWithHeaders() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
HttpHeaders httpHeaders = mock(HttpHeaders.class); HttpHeaders httpHeaders = mock(HttpHeaders.class);
expectAndCallbackNotLeaderException(cb) expectAndCallbackNotLeaderException(cb)
@ -371,7 +369,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorExists() { public void testCreateConnectorExists() {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME, 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); final ArgumentCaptor<Callback<Herder.Created<ConnectorInfo>>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackException(cb, new AlreadyExistsException("already exists")) expectAndCallbackException(cb, new AlreadyExistsException("already exists"))
@ -496,7 +494,7 @@ public class ConnectorsResourceTest {
connectorTask0Configs.put("connector-task1-config0", "321"); connectorTask0Configs.put("connector-task1-config0", "321");
connectorTask0Configs.put("connector-task1-config1", "654"); connectorTask0Configs.put("connector-task1-config1", "654");
final ConnectorTaskId connector2Task0 = new ConnectorTaskId(CONNECTOR2_NAME, 0); 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<>(); final List<TaskInfo> expectedTasksConnector = new ArrayList<>();
expectedTasksConnector.add(new TaskInfo(connectorTask0, connectorTask0Configs)); expectedTasksConnector.add(new TaskInfo(connectorTask0, connectorTask0Configs));
@ -529,7 +527,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithSpecialCharsInName() throws Throwable { public void testCreateConnectorWithSpecialCharsInName() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME_SPECIAL_CHARS, 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); 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, expectAndCallbackResult(cb, new Herder.Created<>(true, new ConnectorInfo(CONNECTOR_NAME_SPECIAL_CHARS, CONNECTOR_CONFIG,
@ -544,7 +542,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testCreateConnectorWithControlSequenceInName() throws Throwable { public void testCreateConnectorWithControlSequenceInName() throws Throwable {
CreateConnectorRequest body = new CreateConnectorRequest(CONNECTOR_NAME_CONTROL_SEQUENCES1, 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); 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, 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( ConnectorStateInfo.ConnectorState state = new ConnectorStateInfo.ConnectorState(
AbstractStatus.State.RESTARTING.name(), "foo", null, null 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); RestartRequest restartRequest = new RestartRequest(CONNECTOR_NAME, true, false);
final ArgumentCaptor<Callback<ConnectorStateInfo>> cb = ArgumentCaptor.forClass(Callback.class); final ArgumentCaptor<Callback<ConnectorStateInfo>> cb = ArgumentCaptor.forClass(Callback.class);
@ -859,9 +857,9 @@ public class ConnectorsResourceTest {
@Test @Test
public void testGetOffsets() throws Throwable { public void testGetOffsets() throws Throwable {
final ArgumentCaptor<Callback<ConnectorOffsets>> cb = ArgumentCaptor.forClass(Callback.class); final ArgumentCaptor<Callback<ConnectorOffsets>> cb = ArgumentCaptor.forClass(Callback.class);
ConnectorOffsets offsets = new ConnectorOffsets(Arrays.asList( ConnectorOffsets offsets = new ConnectorOffsets(List.of(
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue"), Collections.singletonMap("offsetKey", "offsetValue")), new ConnectorOffset(Map.of("partitionKey", "partitionValue"), Map.of("offsetKey", "offsetValue")),
new ConnectorOffset(Collections.singletonMap("partitionKey", "partitionValue2"), Collections.singletonMap("offsetKey", "offsetValue")) new ConnectorOffset(Map.of("partitionKey", "partitionValue2"), Map.of("offsetKey", "offsetValue"))
)); ));
expectAndCallbackResult(cb, offsets).when(herder).connectorOffsets(eq(CONNECTOR_NAME), cb.capture()); expectAndCallbackResult(cb, offsets).when(herder).connectorOffsets(eq(CONNECTOR_NAME), cb.capture());
@ -871,7 +869,7 @@ public class ConnectorsResourceTest {
@Test @Test
public void testAlterOffsetsEmptyOffsets() { public void testAlterOffsetsEmptyOffsets() {
assertThrows(BadRequestException.class, () -> connectorsResource.alterConnectorOffsets( assertThrows(BadRequestException.class, () -> connectorsResource.alterConnectorOffsets(
false, NULL_HEADERS, CONNECTOR_NAME, new ConnectorOffsets(Collections.emptyList()))); false, NULL_HEADERS, CONNECTOR_NAME, new ConnectorOffsets(List.of())));
} }
@Test @Test
@ -879,7 +877,7 @@ public class ConnectorsResourceTest {
Map<String, ?> partition = new HashMap<>(); Map<String, ?> partition = new HashMap<>();
Map<String, ?> offset = new HashMap<>(); Map<String, ?> offset = new HashMap<>();
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset); 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); final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackNotLeaderException(cb).when(herder).alterConnectorOffsets(eq(CONNECTOR_NAME), eq(body.toMap()), cb.capture()); 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, ?> partition = new HashMap<>();
Map<String, ?> offset = new HashMap<>(); Map<String, ?> offset = new HashMap<>();
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset); 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); final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
expectAndCallbackException(cb, new NotFoundException("Connector not found")) expectAndCallbackException(cb, new NotFoundException("Connector not found"))
.when(herder).alterConnectorOffsets(eq(CONNECTOR_NAME), eq(body.toMap()), cb.capture()); .when(herder).alterConnectorOffsets(eq(CONNECTOR_NAME), eq(body.toMap()), cb.capture());
@ -904,10 +902,10 @@ public class ConnectorsResourceTest {
@Test @Test
public void testAlterOffsets() throws Throwable { public void testAlterOffsets() throws Throwable {
Map<String, ?> partition = Collections.singletonMap("partitionKey", "partitionValue"); Map<String, ?> partition = Map.of("partitionKey", "partitionValue");
Map<String, ?> offset = Collections.singletonMap("offsetKey", "offsetValue"); Map<String, ?> offset = Map.of("offsetKey", "offsetValue");
ConnectorOffset connectorOffset = new ConnectorOffset(partition, offset); 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); final ArgumentCaptor<Callback<Message>> cb = ArgumentCaptor.forClass(Callback.class);
Message msg = new Message("The offsets for this connector have been altered successfully"); 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.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64; import java.util.Base64;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -66,8 +65,8 @@ public class InternalConnectResourceTest {
private static final HttpHeaders NULL_HEADERS = null; private static final HttpHeaders NULL_HEADERS = null;
private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>(); private static final List<Map<String, String>> TASK_CONFIGS = new ArrayList<>();
static { static {
TASK_CONFIGS.add(Collections.singletonMap("config", "value")); TASK_CONFIGS.add(Map.of("config", "value"));
TASK_CONFIGS.add(Collections.singletonMap("config", "other_value")); TASK_CONFIGS.add(Map.of("config", "other_value"));
} }
private static final String FENCE_PATH = "/connectors/" + CONNECTOR_NAME + "/fence"; private static final String FENCE_PATH = "/connectors/" + CONNECTOR_NAME + "/fence";
private static final String TASK_CONFIGS_PATH = "/connectors/" + CONNECTOR_NAME + "/tasks"; 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 org.slf4j.event.Level;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map;
import jakarta.ws.rs.core.Response; import jakarta.ws.rs.core.Response;
@ -90,7 +90,7 @@ public class LoggingResourceTest {
BadRequestException.class, BadRequestException.class,
() -> loggingResource.setLevel( () -> loggingResource.setLevel(
"@root", "@root",
Collections.emptyMap(), Map.of(),
scope scope
) )
); );
@ -104,7 +104,7 @@ public class LoggingResourceTest {
NotFoundException.class, NotFoundException.class,
() -> loggingResource.setLevel( () -> loggingResource.setLevel(
"@root", "@root",
Collections.singletonMap("level", "HIGH"), Map.of("level", "HIGH"),
scope scope
) )
); );
@ -130,7 +130,7 @@ public class LoggingResourceTest {
private void testSetLevelWorkerScope(String scope, boolean expectWarning) { private void testSetLevelWorkerScope(String scope, boolean expectWarning) {
final String logger = "org.apache.kafka.connect"; final String logger = "org.apache.kafka.connect";
final String level = "TRACE"; final String level = "TRACE";
final List<String> expectedLoggers = Arrays.asList( final List<String> expectedLoggers = List.of(
"org.apache.kafka.connect", "org.apache.kafka.connect",
"org.apache.kafka.connect.runtime.distributed.DistributedHerder" "org.apache.kafka.connect.runtime.distributed.DistributedHerder"
); );
@ -138,7 +138,7 @@ public class LoggingResourceTest {
List<String> actualLoggers; List<String> actualLoggers;
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(LoggingResource.class)) { 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()); assertEquals(Response.Status.OK.getStatusCode(), response.getStatus());
actualLoggers = (List<String>) response.getEntity(); actualLoggers = (List<String>) response.getEntity();
long warningMessages = logCaptureAppender.getEvents().stream() long warningMessages = logCaptureAppender.getEvents().stream()
@ -159,7 +159,7 @@ public class LoggingResourceTest {
final String logger = "org.apache.kafka.connect"; final String logger = "org.apache.kafka.connect";
final String level = "TRACE"; 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()); assertEquals(Response.Status.NO_CONTENT.getStatusCode(), response.getStatus());
assertNull(response.getEntity()); 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.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertArrayEquals; 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_KEYSTORE_TYPE, ssl.getKeyStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType()); assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, ssl.getProtocol()); 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_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm()); assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm());
assertFalse(ssl.getNeedClientAuth()); assertFalse(ssl.getNeedClientAuth());
@ -181,7 +181,7 @@ public class SSLUtilsTest {
assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType()); assertEquals(SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ssl.getKeyStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType()); assertEquals(SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE, ssl.getTrustStoreType());
assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, ssl.getProtocol()); 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_KEYMANGER_ALGORITHM, ssl.getKeyManagerFactoryAlgorithm());
assertEquals(SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, ssl.getTrustManagerFactoryAlgorithm()); 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.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; 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.DEFAULT_TOPIC_CREATION_PREFIX;
import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG;
import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_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()); when(connectorMock.config()).thenReturn(new ConfigDef());
ConfigValue validatedValue = new ConfigValue("foo.bar"); 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); herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
@ -265,7 +261,7 @@ public class StandaloneHerderTest {
herder.putConnectorConfig(CONNECTOR_NAME, config, TargetState.STOPPED, false, createCallback); herder.putConnectorConfig(CONNECTOR_NAME, config, TargetState.STOPPED, false, createCallback);
Herder.Created<ConnectorInfo> connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); Herder.Created<ConnectorInfo> connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
assertEquals( 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() connectorInfo.result()
); );
verify(loaderSwap).close(); verify(loaderSwap).close();
@ -279,7 +275,7 @@ public class StandaloneHerderTest {
Map<String, String> config = connectorConfig(SourceSink.SOURCE); Map<String, String> config = connectorConfig(SourceSink.SOURCE);
expectConfigValidation(SourceSink.SOURCE, config); 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); when(worker.connectorVersion(CONNECTOR_NAME)).thenReturn(null);
herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback); herder.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
@ -315,11 +311,11 @@ public class StandaloneHerderTest {
mockStartConnector(config, TargetState.STARTED, TargetState.STARTED, null); 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); when(worker.getPlugins()).thenReturn(plugins);
// same task configs as earlier, so don't expect a new set of tasks to be brought up // 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))) 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.putConnectorConfig(CONNECTOR_NAME, config, false, createCallback);
Herder.Created<ConnectorInfo> connectorInfo = createCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); 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); 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); 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) // 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); Map<String, String> taskConfigs = taskConfig(SourceSink.SOURCE);
taskConfigs.put("k", "v"); taskConfigs.put("k", "v");
when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, config, true))) 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); 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( ClusterConfigState configState = new ClusterConfigState(
-1, -1,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 1), Map.of(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Map.of(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Map.of(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SOURCE)), Map.of(taskId, taskConfig(SourceSink.SOURCE)),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(), new HashSet<>(),
new HashSet<>(), new HashSet<>(),
transformer); transformer);
@ -436,13 +432,13 @@ public class StandaloneHerderTest {
ClusterConfigState configState = new ClusterConfigState( ClusterConfigState configState = new ClusterConfigState(
-1, -1,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 1), Map.of(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Map.of(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Map.of(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)), Map.of(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(), new HashSet<>(),
new HashSet<>(), new HashSet<>(),
transformer); transformer);
@ -560,7 +556,7 @@ public class StandaloneHerderTest {
when(restartPlan.shouldRestartTasks()).thenReturn(true); when(restartPlan.shouldRestartTasks()).thenReturn(true);
when(restartPlan.restartTaskCount()).thenReturn(1); when(restartPlan.restartTaskCount()).thenReturn(1);
when(restartPlan.totalTaskCount()).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); when(restartPlan.restartConnectorStateInfo()).thenReturn(connectorStateInfo);
doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest);
@ -570,18 +566,18 @@ public class StandaloneHerderTest {
Map<String, String> connectorConfig = connectorConfig(SourceSink.SINK); Map<String, String> connectorConfig = connectorConfig(SourceSink.SINK);
expectConfigValidation(SourceSink.SINK, connectorConfig); expectConfigValidation(SourceSink.SINK, connectorConfig);
doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId)); doNothing().when(worker).stopAndAwaitTasks(List.of(taskId));
ClusterConfigState configState = new ClusterConfigState( ClusterConfigState configState = new ClusterConfigState(
-1, -1,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 1), Map.of(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Map.of(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Map.of(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)), Map.of(taskId, taskConfig(SourceSink.SINK)),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(), new HashSet<>(),
new HashSet<>(), new HashSet<>(),
transformer); transformer);
@ -612,7 +608,7 @@ public class StandaloneHerderTest {
when(restartPlan.shouldRestartTasks()).thenReturn(true); when(restartPlan.shouldRestartTasks()).thenReturn(true);
when(restartPlan.restartTaskCount()).thenReturn(1); when(restartPlan.restartTaskCount()).thenReturn(1);
when(restartPlan.totalTaskCount()).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); when(restartPlan.restartConnectorStateInfo()).thenReturn(connectorStateInfo);
doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest); doReturn(Optional.of(restartPlan)).when(herder).buildRestartPlan(restartRequest);
@ -626,20 +622,20 @@ public class StandaloneHerderTest {
expectConfigValidation(SourceSink.SINK, connectorConfig); expectConfigValidation(SourceSink.SINK, connectorConfig);
doNothing().when(worker).stopAndAwaitConnector(CONNECTOR_NAME); 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); mockStartConnector(connectorConfig, null, TargetState.STARTED, null);
ClusterConfigState configState = new ClusterConfigState( ClusterConfigState configState = new ClusterConfigState(
-1, -1,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 1), Map.of(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Map.of(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Map.of(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(taskId, taskConfig(SourceSink.SINK)), Map.of(taskId, taskConfig(SourceSink.SINK)),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(), new HashSet<>(),
new HashSet<>(), new HashSet<>(),
transformer); transformer);
@ -694,7 +690,7 @@ public class StandaloneHerderTest {
Callback<List<TaskInfo>> taskConfigsCb = mock(Callback.class); Callback<List<TaskInfo>> taskConfigsCb = mock(Callback.class);
// Check accessors with empty worker // 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(connectorInfoCb).onCompletion(any(NotFoundException.class), isNull());
doNothing().when(taskConfigsCb).onCompletion(any(NotFoundException.class), isNull()); doNothing().when(taskConfigsCb).onCompletion(any(NotFoundException.class), isNull());
doNothing().when(connectorConfigCb).onCompletion(any(NotFoundException.class), isNull()); doNothing().when(connectorConfigCb).onCompletion(any(NotFoundException.class), isNull());
@ -703,13 +699,13 @@ public class StandaloneHerderTest {
expectConfigValidation(SourceSink.SOURCE, connConfig); expectConfigValidation(SourceSink.SOURCE, connConfig);
// Validate accessors with 1 connector // Validate accessors with 1 connector
doNothing().when(listConnectorsCb).onCompletion(null, singleton(CONNECTOR_NAME)); doNothing().when(listConnectorsCb).onCompletion(null, Set.of(CONNECTOR_NAME));
ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, singletonList(new ConnectorTaskId(CONNECTOR_NAME, 0)), ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, List.of(new ConnectorTaskId(CONNECTOR_NAME, 0)),
ConnectorType.SOURCE); ConnectorType.SOURCE);
doNothing().when(connectorInfoCb).onCompletion(null, connInfo); doNothing().when(connectorInfoCb).onCompletion(null, connInfo);
TaskInfo taskInfo = new TaskInfo(new ConnectorTaskId(CONNECTOR_NAME, 0), taskConfig(SourceSink.SOURCE)); 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 // All operations are synchronous for StandaloneHerder, so we don't need to actually wait after making each call
herder.connectors(listConnectorsCb); herder.connectors(listConnectorsCb);
@ -758,8 +754,8 @@ public class StandaloneHerderTest {
// Generate same task config, but from different connector config, resulting // Generate same task config, but from different connector config, resulting
// in task restarts // in task restarts
when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig, true))) when(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig, true)))
.thenReturn(singletonList(taskConfig(SourceSink.SOURCE))); .thenReturn(List.of(taskConfig(SourceSink.SOURCE)));
doNothing().when(worker).stopAndAwaitTasks(Collections.singletonList(taskId)); doNothing().when(worker).stopAndAwaitTasks(List.of(taskId));
doNothing().when(statusBackingStore).put(new TaskStatus(taskId, TaskStatus.State.DESTROYED, WORKER_ID, 0)); 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); 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); doNothing().when(connectorConfigCb).onCompletion(null, newConnConfig);
herder.putConnectorConfig(CONNECTOR_NAME, newConnConfig, true, reconfigureCallback); herder.putConnectorConfig(CONNECTOR_NAME, newConnConfig, true, reconfigureCallback);
Herder.Created<ConnectorInfo> newConnectorInfo = reconfigureCallback.get(1000L, TimeUnit.SECONDS); 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); ConnectorType.SOURCE);
assertEquals(newConnInfo, newConnectorInfo.result()); assertEquals(newConnInfo, newConnectorInfo.result());
@ -852,7 +848,7 @@ public class StandaloneHerderTest {
eq(herder), eq(TargetState.STARTED), onStart.capture()); eq(herder), eq(TargetState.STARTED), onStart.capture());
ConnectorConfig connConfig = new SourceConnectorConfig(plugins, config, true); ConnectorConfig connConfig = new SourceConnectorConfig(plugins, config, true);
when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig)) when(worker.connectorTaskConfigs(CONNECTOR_NAME, connConfig))
.thenReturn(emptyList()); .thenReturn(List.of());
} }
@Test @Test
@ -861,7 +857,7 @@ public class StandaloneHerderTest {
Callback<Void> cb = mock(Callback.class); Callback<Void> cb = mock(Callback.class);
assertThrows(UnsupportedOperationException.class, () -> herder.putTaskConfigs(CONNECTOR_NAME, assertThrows(UnsupportedOperationException.class, () -> herder.putTaskConfigs(CONNECTOR_NAME,
singletonList(singletonMap("config", "value")), cb, null)); List.of(Map.of("config", "value")), cb, null));
} }
@Test @Test
@ -874,11 +870,11 @@ public class StandaloneHerderTest {
config.put(SinkConnectorConfig.TOPICS_CONFIG, TOPICS_LIST_STR); config.put(SinkConnectorConfig.TOPICS_CONFIG, TOPICS_LIST_STR);
Connector connectorMock = mock(SinkConnector.class); Connector connectorMock = mock(SinkConnector.class);
String error = "This is an error in your config!"; 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"; String key = "foo.invalid.key";
when(connectorMock.validate(config)).thenReturn( when(connectorMock.validate(config)).thenReturn(
new Config( new Config(
singletonList(new ConfigValue(key, null, Collections.emptyList(), errors)) List.of(new ConfigValue(key, null, List.of(), errors))
) )
); );
ConfigDef configDef = new ConfigDef(); 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)); verify(statusBackingStore).put(new TaskStatus(new ConnectorTaskId(CONNECTOR_NAME, 0), AbstractStatus.State.DESTROYED, WORKER_ID, 0));
stopCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); stopCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS);
herder.taskConfigs(CONNECTOR_NAME, taskConfigsCallback); 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() should stop any running connectors and tasks even if destroyConnector was not invoked
herder.stop(); herder.stop();
@ -953,7 +949,7 @@ public class StandaloneHerderTest {
initialize(false); initialize(false);
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>(); FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets("unknown-connector", 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); alterOffsetsCallback);
ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS)); ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
assertInstanceOf(NotFoundException.class, e.getCause()); assertInstanceOf(NotFoundException.class, e.getCause());
@ -972,20 +968,20 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState( herder.configState = new ClusterConfigState(
10, 10,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 3), Map.of(CONNECTOR_NAME, 3),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)), Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.PAUSED), Map.of(CONNECTOR_NAME, TargetState.PAUSED),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(), Set.of(),
Collections.emptySet() Set.of()
); );
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>(); FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets(CONNECTOR_NAME, 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); alterOffsetsCallback);
ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS)); ExecutionException e = assertThrows(ExecutionException.class, () -> alterOffsetsCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS));
assertInstanceOf(BadRequestException.class, e.getCause()); assertInstanceOf(BadRequestException.class, e.getCause());
@ -1011,19 +1007,19 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState( herder.configState = new ClusterConfigState(
10, 10,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 0), Map.of(CONNECTOR_NAME, 0),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)), Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STOPPED), Map.of(CONNECTOR_NAME, TargetState.STOPPED),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(), Set.of(),
Collections.emptySet() Set.of()
); );
FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>(); FutureCallback<Message> alterOffsetsCallback = new FutureCallback<>();
herder.alterConnectorOffsets(CONNECTOR_NAME, 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); alterOffsetsCallback);
assertEquals(msg, alterOffsetsCallback.get(1000, TimeUnit.MILLISECONDS)); assertEquals(msg, alterOffsetsCallback.get(1000, TimeUnit.MILLISECONDS));
} }
@ -1044,15 +1040,15 @@ public class StandaloneHerderTest {
herder.configState = new ClusterConfigState( herder.configState = new ClusterConfigState(
10, 10,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 0), Map.of(CONNECTOR_NAME, 0),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)), Map.of(CONNECTOR_NAME, connectorConfig(SourceSink.SOURCE)),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STOPPED), Map.of(CONNECTOR_NAME, TargetState.STOPPED),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
Collections.emptySet(), Set.of(),
Collections.emptySet() Set.of()
); );
FutureCallback<Message> resetOffsetsCallback = new FutureCallback<>(); FutureCallback<Message> resetOffsetsCallback = new FutureCallback<>();
herder.resetConnectorOffsets(CONNECTOR_NAME, resetOffsetsCallback); herder.resetConnectorOffsets(CONNECTOR_NAME, resetOffsetsCallback);
@ -1078,7 +1074,7 @@ public class StandaloneHerderTest {
assertEquals(createdInfo(SourceSink.SOURCE), connectorInfo.result()); assertEquals(createdInfo(SourceSink.SOURCE), connectorInfo.result());
// Prepare for task config update // Prepare for task config update
when(worker.connectorNames()).thenReturn(Collections.singleton(CONNECTOR_NAME)); when(worker.connectorNames()).thenReturn(Set.of(CONNECTOR_NAME));
expectStop(); expectStop();
@ -1090,8 +1086,8 @@ public class StandaloneHerderTest {
updatedTaskConfig2.put("dummy-task-property", "2"); updatedTaskConfig2.put("dummy-task-property", "2");
when(worker.connectorTaskConfigs(eq(CONNECTOR_NAME), any())) when(worker.connectorTaskConfigs(eq(CONNECTOR_NAME), any()))
.thenReturn( .thenReturn(
Collections.singletonList(updatedTaskConfig1), List.of(updatedTaskConfig1),
Collections.singletonList(updatedTaskConfig2)); List.of(updatedTaskConfig2));
// Set new config on the connector and tasks // Set new config on the connector and tasks
FutureCallback<Herder.Created<ConnectorInfo>> reconfigureCallback = new FutureCallback<>(); FutureCallback<Herder.Created<ConnectorInfo>> reconfigureCallback = new FutureCallback<>();
@ -1102,7 +1098,7 @@ public class StandaloneHerderTest {
// Wait on connector update // Wait on connector update
Herder.Created<ConnectorInfo> updatedConnectorInfo = reconfigureCallback.get(WAIT_TIME_MS, TimeUnit.MILLISECONDS); 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()); assertEquals(expectedConnectorInfo, updatedConnectorInfo.result());
verify(statusBackingStore, times(2)).put(new TaskStatus(new ConnectorTaskId(CONNECTOR_NAME, 0), TaskStatus.State.DESTROYED, WORKER_ID, 0)); 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); Map<String, String> generatedTaskProps = taskConfig(sourceSink);
if (mockConnectorTaskConfigs) { 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( ClusterConfigState configState = new ClusterConfigState(
-1, -1,
null, null,
Collections.singletonMap(CONNECTOR_NAME, 1), Map.of(CONNECTOR_NAME, 1),
Collections.singletonMap(CONNECTOR_NAME, connectorConfig), Map.of(CONNECTOR_NAME, connectorConfig),
Collections.singletonMap(CONNECTOR_NAME, TargetState.STARTED), Map.of(CONNECTOR_NAME, TargetState.STARTED),
Collections.singletonMap(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps), Map.of(new ConnectorTaskId(CONNECTOR_NAME, 0), generatedTaskProps),
Collections.emptyMap(), Map.of(),
Collections.emptyMap(), Map.of(),
Collections.singletonMap(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)), Map.of(CONNECTOR_NAME, new AppliedConnectorConfig(connectorConfig)),
new HashSet<>(), new HashSet<>(),
new HashSet<>(), new HashSet<>(),
transformer); transformer);
@ -1192,13 +1188,13 @@ public class StandaloneHerderTest {
private ConnectorInfo createdInfo(SourceSink sourceSink) { private ConnectorInfo createdInfo(SourceSink sourceSink) {
return new ConnectorInfo(CONNECTOR_NAME, connectorConfig(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); SourceSink.SOURCE == sourceSink ? ConnectorType.SOURCE : ConnectorType.SINK);
} }
private void expectStop() { private void expectStop() {
ConnectorTaskId task = new ConnectorTaskId(CONNECTOR_NAME, 0); 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); doNothing().when(worker).stopAndAwaitConnector(CONNECTOR_NAME);
} }
@ -1253,7 +1249,7 @@ public class StandaloneHerderTest {
// Set up validation for each config // Set up validation for each config
for (Map<String, String> config : configs) { 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 org.mockito.quality.Strictness;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future; import java.util.concurrent.Future;
@ -440,7 +440,7 @@ public class ConnectorOffsetBackingStoreTest {
MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(AutoOffsetResetStrategy.LATEST.name()); MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(AutoOffsetResetStrategy.LATEST.name());
Node noNode = Node.noNode(); Node noNode = Node.noNode();
Node[] nodes = new 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))); consumer.updateBeginningOffsets(mkMap(mkEntry(new TopicPartition(topic, 0), 100L)));
return consumer; return consumer;
} }

View File

@ -34,8 +34,6 @@ import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.nio.file.Files; import java.nio.file.Files;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -79,7 +77,7 @@ public class FileOffsetBackingStoreTest {
converter = mock(Converter.class); converter = mock(Converter.class);
// This is only needed for storing deserialized connector partitions, which we don't test in most of the cases here // 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, 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); store = new FileOffsetBackingStore(converter);
tempFile = assertDoesNotThrow(() -> File.createTempFile("fileoffsetbackingstore", null)); tempFile = assertDoesNotThrow(() -> File.createTempFile("fileoffsetbackingstore", null));
Map<String, String> props = new HashMap<>(); Map<String, String> props = new HashMap<>();
@ -105,7 +103,7 @@ public class FileOffsetBackingStoreTest {
store.set(FIRST_SET, setCallback).get(); 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"))); assertEquals(buffer("value"), values.get(buffer("key")));
assertNull(values.get(buffer("bad"))); assertNull(values.get(buffer("bad")));
verify(setCallback).onCompletion(isNull(), isNull()); verify(setCallback).onCompletion(isNull(), isNull());
@ -123,7 +121,7 @@ public class FileOffsetBackingStoreTest {
FileOffsetBackingStore restore = new FileOffsetBackingStore(converter); FileOffsetBackingStore restore = new FileOffsetBackingStore(converter);
restore.configure(config); restore.configure(config);
restore.start(); 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"))); assertEquals(buffer("value"), values.get(buffer("key")));
verify(setCallback).onCompletion(isNull(), isNull()); 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 // 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 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<>(); Map<ByteBuffer, ByteBuffer> serializedPartitionOffsets = new HashMap<>();
serializedPartitionOffsets.put( serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")), serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue")) serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
); );
store.set(serializedPartitionOffsets, setCallback).get(); store.set(serializedPartitionOffsets, setCallback).get();
serializedPartitionOffsets.put( serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")), serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue2")) serialize(jsonConverter, Map.of("offsetKey", "offsetValue2"))
); );
serializedPartitionOffsets.put( serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue2")), serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue2")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue")) serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
); );
serializedPartitionOffsets.put( serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector2", Collections.singletonMap("partitionKey", "partitionValue")), serializeKey(jsonConverter, "connector2", Map.of("partitionKey", "partitionValue")),
serialize(jsonConverter, Collections.singletonMap("offsetKey", "offsetValue")) serialize(jsonConverter, Map.of("offsetKey", "offsetValue"))
); );
store.set(serializedPartitionOffsets, setCallback).get(); store.set(serializedPartitionOffsets, setCallback).get();
@ -167,23 +165,23 @@ public class FileOffsetBackingStoreTest {
Set<Map<String, Object>> connectorPartitions1 = restore.connectorPartitions("connector1"); Set<Map<String, Object>> connectorPartitions1 = restore.connectorPartitions("connector1");
Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>(); Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue1")); expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue2")); expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue2"));
assertEquals(expectedConnectorPartition1, connectorPartitions1); assertEquals(expectedConnectorPartition1, connectorPartitions1);
Set<Map<String, Object>> connectorPartitions2 = restore.connectorPartitions("connector2"); 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); assertEquals(expectedConnectorPartition2, connectorPartitions2);
serializedPartitionOffsets.clear(); serializedPartitionOffsets.clear();
// Null valued offset for a partition key should remove that partition for the connector // Null valued offset for a partition key should remove that partition for the connector
serializedPartitionOffsets.put( serializedPartitionOffsets.put(
serializeKey(jsonConverter, "connector1", Collections.singletonMap("partitionKey", "partitionValue1")), serializeKey(jsonConverter, "connector1", Map.of("partitionKey", "partitionValue1")),
null null
); );
restore.set(serializedPartitionOffsets, setCallback).get(); restore.set(serializedPartitionOffsets, setCallback).get();
connectorPartitions1 = restore.connectorPartitions("connector1"); 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()); 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) { 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); return serialize(converter, nameAndPartition);
} }

View File

@ -61,8 +61,6 @@ import org.mockito.stubbing.Answer;
import java.time.Duration; import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.LinkedHashMap; 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"); 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_IDS = List.of("connector1", "connector2");
private static final List<String> CONNECTOR_CONFIG_KEYS = Arrays.asList("connector-connector1", "connector-connector2"); private static final List<String> CONNECTOR_CONFIG_KEYS = List.of("connector-connector1", "connector-connector2");
private static final List<String> COMMIT_TASKS_CONFIG_KEYS = Arrays.asList("commit-connector1", "commit-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> TARGET_STATE_KEYS = List.of("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> 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_1_NAME = "connector1";
private static final String CONNECTOR_2_NAME = "connector2"; 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 // 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", 0),
new ConnectorTaskId("connector1", 1), new ConnectorTaskId("connector1", 1),
new ConnectorTaskId("connector2", 0) 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 // 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( private static final List<Map<String, String>> SAMPLE_CONFIGS = List.of(
Collections.singletonMap("config-key-one", "config-value-one"), Map.of("config-key-one", "config-value-one"),
Collections.singletonMap("config-key-two", "config-value-two"), Map.of("config-key-two", "config-value-two"),
Collections.singletonMap("config-key-three", "config-value-three") 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(0)),
new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)) 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 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 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), new Struct(KafkaConfigBackingStore.RESTART_REQUEST_V0).put(ONLY_FAILED_FIELD_NAME, true).put(INCLUDE_TASKS_FIELD_NAME, false),
ONLY_FAILED_MISSING_STRUCT, ONLY_FAILED_MISSING_STRUCT,
INCLUDE_TASKS_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(0)),
new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)), new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)),
new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2)) 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) private static final Struct TARGET_STATE_STOPPED = new Struct(KafkaConfigBackingStore.TARGET_STATE_V1)
.put("state", "PAUSED") .put("state", "PAUSED")
.put("state.v2", "STOPPED"); .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", 6),
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9), new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 9),
new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 2) new Struct(KafkaConfigBackingStore.TASK_COUNT_RECORD_V0).put("task-count", 2)
); );
// The exact format doesn't matter here since both conversions are mocked // 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-1".getBytes(), "config-bytes-2".getBytes(), "config-bytes-3".getBytes(),
"config-bytes-4".getBytes(), "config-bytes-5".getBytes(), "config-bytes-6".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() "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 private static final Struct TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR
= new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 0); = 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() "started".getBytes(), "paused".getBytes(), "stopped".getBytes()
); );
@Mock @Mock
@ -308,8 +306,8 @@ public class KafkaConfigBackingStoreTest {
String configKey = CONNECTOR_CONFIG_KEYS.get(1); String configKey = CONNECTOR_CONFIG_KEYS.get(1);
String targetStateKey = TARGET_STATE_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(Map.of(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(1), CONFIGS_SERIALIZED.get(1))))
// Config deletion // Config deletion
.doAnswer(expectReadToEnd(new LinkedHashMap<>() {{ .doAnswer(expectReadToEnd(new LinkedHashMap<>() {{
put(configKey, null); put(configKey, null);
@ -426,7 +424,7 @@ public class KafkaConfigBackingStoreTest {
assertEquals(-1, configState.offset()); assertEquals(-1, configState.offset());
assertEquals(0, configState.connectors().size()); 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); when(producerFuture.get(anyLong(), any(TimeUnit.class))).thenThrow(thrownException);
// verify that the producer exception from KafkaBasedLog::send is propagated // verify that the producer exception from KafkaBasedLog::send is propagated
@ -510,8 +508,8 @@ public class KafkaConfigBackingStoreTest {
doReturn(fencableProducer).when(configStorage).createFencableProducer(); doReturn(fencableProducer).when(configStorage).createFencableProducer();
// And write the task count record successfully // And write the task count record successfully
when(fencableProducer.send(any(ProducerRecord.class))).thenReturn(null); when(fencableProducer.send(any(ProducerRecord.class))).thenReturn(null);
doAnswer(expectReadToEnd(Collections.singletonMap(CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)))) doAnswer(expectReadToEnd(Map.of(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(1), CONFIGS_SERIALIZED.get(2))))
.when(configLog).readToEnd(); .when(configLog).readToEnd();
when(converter.toConnectData(TOPIC, CONFIGS_SERIALIZED.get(0))) when(converter.toConnectData(TOPIC, CONFIGS_SERIALIZED.get(0)))
.thenReturn(new SchemaAndValue(null, structToMap(CONNECTOR_TASK_COUNT_RECORD_STRUCTS.get(0)))); .thenReturn(new SchemaAndValue(null, structToMap(CONNECTOR_TASK_COUNT_RECORD_STRUCTS.get(0))));
@ -570,7 +568,7 @@ public class KafkaConfigBackingStoreTest {
@Test @Test
public void testRestoreTargetStateUnexpectedDeletion() { 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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 // The target state deletion should reset the state to STARTED
ClusterConfigState configState = configStorage.snapshot(); ClusterConfigState configState = configStorage.snapshot();
assertEquals(5, configState.offset()); // Should always be next to be read, even if uncommitted 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))); assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
configStorage.stop(); configStorage.stop();
@ -608,7 +606,7 @@ public class KafkaConfigBackingStoreTest {
@Test @Test
public void testRestoreTargetState() { 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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 // Should see a single connector with initial state paused
ClusterConfigState configState = configStorage.snapshot(); ClusterConfigState configState = configStorage.snapshot();
assertEquals(6, configState.offset()); // Should always be next to be read, even if uncommitted 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.PAUSED, configState.targetState(CONNECTOR_IDS.get(0)));
assertEquals(TargetState.STOPPED, configState.targetState(CONNECTOR_IDS.get(1))); assertEquals(TargetState.STOPPED, configState.targetState(CONNECTOR_IDS.get(1)));
@ -657,7 +655,7 @@ public class KafkaConfigBackingStoreTest {
// that inconsistent state is ignored. // that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading // 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), 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()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), 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 // Should see a single connector and its config should be the last one seen anywhere in the log
ClusterConfigState configState = configStorage.snapshot(); ClusterConfigState configState = configStorage.snapshot();
assertEquals(logOffset, configState.offset()); // Should always be next to be read, even if uncommitted 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))); assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
// CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2] // CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0))); 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 // 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] // 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(0)));
assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(1))); assertEquals(SAMPLE_CONFIGS.get(0), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(9, (int) configState.taskCountRecord(CONNECTOR_IDS.get(1))); assertEquals(9, (int) configState.taskCountRecord(CONNECTOR_IDS.get(1)));
assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); assertEquals(Set.of(), configState.inconsistentConnectors());
assertEquals(Collections.singleton("connector1"), configState.connectorsPendingFencing); assertEquals(Set.of("connector1"), configState.connectorsPendingFencing);
// Shouldn't see any callbacks since this is during startup // Shouldn't see any callbacks since this is during startup
configStorage.stop(); configStorage.stop();
@ -723,7 +721,7 @@ public class KafkaConfigBackingStoreTest {
// that inconsistent state is ignored. // that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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. // that inconsistent state is ignored.
// Overwrite each type at least once to ensure we see the latest data after loading // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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 // Should see a single connector and its config should be the last one seen anywhere in the log
ClusterConfigState configState = configStorage.snapshot(); ClusterConfigState configState = configStorage.snapshot();
assertEquals(8, configState.offset()); // Should always be next to be read, even if uncommitted 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] // CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2]
assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0))); assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0)));
// Should see 0 tasks for that connector. // 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] // 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 // Shouldn't see any callbacks since this is during startup
configStorage.stop(); configStorage.stop();
@ -1020,7 +1018,7 @@ public class KafkaConfigBackingStoreTest {
@Test @Test
public void testBackgroundConnectorDeletion() throws Exception { public void testBackgroundConnectorDeletion() throws Exception {
// verify that we handle connector deletions correctly when they come up through the log // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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))); assertEquals(0, configState.taskCount(CONNECTOR_IDS.get(0)));
// Ensure that the deleted connector's deferred task updates have been cleaned up // Ensure that the deleted connector's deferred task updates have been cleaned up
// in order to prevent unbounded growth of the map // in order to prevent unbounded growth of the map
assertEquals(Collections.emptyMap(), configStorage.deferredTaskUpdates); assertEquals(Map.of(), configStorage.deferredTaskUpdates);
configStorage.stop(); configStorage.stop();
verify(configLog).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. // 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 // We start out by loading an initial configuration where we started to write a task update, and then
// compaction cleaned up the earlier record. // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
// This is the record that has been compacted: // 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 // After reading the log, it should have been in an inconsistent state
ClusterConfigState configState = configStorage.snapshot(); ClusterConfigState configState = configStorage.snapshot();
assertEquals(6, configState.offset()); // Should always be next to be read, not last committed 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 // 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] // Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0]
assertNull(configState.taskConfig(TASK_IDS.get(0))); assertNull(configState.taskConfig(TASK_IDS.get(0)));
assertNull(configState.taskConfig(TASK_IDS.get(1))); 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 // Records to be read by consumer as it reads to the end of the log
LinkedHashMap<String, byte[]> serializedConfigs = new LinkedHashMap<>(); 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 // 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 // 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 // Validate updated config
configState = configStorage.snapshot(); configState = configStorage.snapshot();
// This is only two more ahead of the last one because multiple calls fail, and so their configs are not written // 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. // to the topic. Only the last call with 1 task config + 1 commit actually gets written.
assertEquals(8, configState.offset()); assertEquals(8, configState.offset());
assertEquals(Collections.singletonList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); assertEquals(List.of(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors()));
assertEquals(Collections.singletonList(TASK_IDS.get(0)), configState.tasks(CONNECTOR_IDS.get(0))); 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(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 // 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(); configStorage.stop();
verify(configLog).stop(); verify(configLog).stop();
@ -1168,7 +1166,7 @@ public class KafkaConfigBackingStoreTest {
} }
private void testPutRestartRequest(RestartRequest restartRequest) { private void testPutRestartRequest(RestartRequest restartRequest) {
expectStart(Collections.emptyList(), Collections.emptyMap()); expectStart(List.of(), Map.of());
when(configLog.partitionCount()).thenReturn(1); when(configLog.partitionCount()).thenReturn(1);
configStorage.setupAndCreateKafkaBasedLog(TOPIC, config); 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 // Restoring data should notify only of the latest values after loading is complete. This also validates
// that inconsistent state doesn't prevent startup. // that inconsistent state doesn't prevent startup.
// Overwrite each type at least once to ensure we see the latest data after loading // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, RESTART_CONNECTOR_KEYS.get(1), 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 // Records to be read by consumer as it reads to the end of the log
doAnswer(expectReadToEnd(new LinkedHashMap<>())). 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(); .when(configLog).readToEnd();
expectConvertWriteRead( expectConvertWriteRead(
@ -1258,7 +1256,7 @@ public class KafkaConfigBackingStoreTest {
"tasks", 0); // We have 0 tasks "tasks", 0); // We have 0 tasks
// Bootstrap as if we had already added the connector, but no tasks had been added yet // 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 // 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 // Writing task configs should block until all the writes have been performed and the root record update
// has completed // has completed
List<Map<String, String>> taskConfigs = Collections.emptyList(); List<Map<String, String>> taskConfigs = List.of();
configStorage.putTaskConfigs("connector1", taskConfigs); configStorage.putTaskConfigs("connector1", taskConfigs);
// Validate root config by listing all connectors and tasks // Validate root config by listing all connectors and tasks
configState = configStorage.snapshot(); configState = configStorage.snapshot();
assertEquals(1, configState.offset()); assertEquals(1, configState.offset());
String connectorName = CONNECTOR_IDS.get(0); String connectorName = CONNECTOR_IDS.get(0);
assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors())); assertEquals(List.of(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(Collections.emptyList(), configState.tasks(connectorName)); assertEquals(List.of(), configState.tasks(connectorName));
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 // 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(); configStorage.stop();
verify(configLog).stop(); verify(configLog).stop();
@ -1288,7 +1286,7 @@ public class KafkaConfigBackingStoreTest {
@Test @Test
public void testBackgroundUpdateTargetState() throws Exception { public void testBackgroundUpdateTargetState() throws Exception {
// verify that we handle target state changes correctly when they come up through the log // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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 // Should see a single connector with initial state started
ClusterConfigState configState = configStorage.snapshot(); 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))); assertEquals(TargetState.STARTED, configState.targetState(CONNECTOR_IDS.get(0)));
LinkedHashMap<String, byte[]> serializedAfterStartup = new LinkedHashMap<>(); LinkedHashMap<String, byte[]> serializedAfterStartup = new LinkedHashMap<>();
@ -1343,7 +1341,7 @@ public class KafkaConfigBackingStoreTest {
@Test @Test
public void testSameTargetState() { public void testSameTargetState() {
// verify that we handle target state changes correctly when they come up through the log // 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), new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, CONNECTOR_CONFIG_KEYS.get(0),
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()), CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty()),
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, TASK_CONFIG_KEYS.get(0), 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., // Pre-populate the config topic with a couple of logger level records; these should be ignored (i.e.,
// not reported to the update listener) // 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, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, "logger-cluster-" + logger1,
CONFIGS_SERIALIZED.get(0), new RecordHeaders(), Optional.empty() 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), 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)); 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 // Before anything is written
String connectorName = CONNECTOR_IDS.get(0); 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 // Writing task configs should block until all the writes have been performed and the root record update
// has completed // 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); configStorage.putTaskConfigs("connector1", taskConfigs);
configState = configStorage.snapshot(); configState = configStorage.snapshot();
@ -1511,7 +1509,7 @@ public class KafkaConfigBackingStoreTest {
assertEquals(0, (long) configState.taskConfigGeneration(connectorName)); assertEquals(0, (long) configState.taskConfigGeneration(connectorName));
// As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks // 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(); configStorage.stop();
verify(configLog).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), 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 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 // 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 // Null before writing
ClusterConfigState configState = configStorage.snapshot(); 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 // Writing task configs should block until all the writes have been performed and the root record update
// has completed // 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); configStorage.putTaskConfigs("connector1", taskConfigs);
// Validate root config by listing all connectors and tasks // Validate root config by listing all connectors and tasks
configState = configStorage.snapshot(); configState = configStorage.snapshot();
assertEquals(3, configState.offset()); assertEquals(3, configState.offset());
String connectorName = CONNECTOR_IDS.get(0); String connectorName = CONNECTOR_IDS.get(0);
assertEquals(Collections.singletonList(connectorName), new ArrayList<>(configState.connectors())); assertEquals(List.of(connectorName), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName)); 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(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1))); 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 // 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(); configStorage.stop();
verify(configLog).stop(); verify(configLog).stop();
@ -1616,8 +1614,8 @@ public class KafkaConfigBackingStoreTest {
assertNull(configState.taskConfig(TASK_IDS.get(1))); assertNull(configState.taskConfig(TASK_IDS.get(1)));
// Bootstrap as if we had already added the connector, but no tasks had been added yet // 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());
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); configStorage.putTaskConfigs("connector1", taskConfigs);
expectConvertWriteRead2( expectConvertWriteRead2(
@ -1627,8 +1625,8 @@ public class KafkaConfigBackingStoreTest {
COMMIT_TASKS_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(4), 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 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()); addConnector(CONNECTOR_IDS.get(1), SAMPLE_CONFIGS.get(1), List.of());
taskConfigs = Collections.singletonList(SAMPLE_CONFIGS.get(2)); taskConfigs = List.of(SAMPLE_CONFIGS.get(2));
configStorage.putTaskConfigs("connector2", taskConfigs); configStorage.putTaskConfigs("connector2", taskConfigs);
// Validate root config by listing all connectors and tasks // Validate root config by listing all connectors and tasks
@ -1636,17 +1634,17 @@ public class KafkaConfigBackingStoreTest {
assertEquals(5, configState.offset()); assertEquals(5, configState.offset());
String connectorName1 = CONNECTOR_IDS.get(0); String connectorName1 = CONNECTOR_IDS.get(0);
String connectorName2 = CONNECTOR_IDS.get(1); String connectorName2 = CONNECTOR_IDS.get(1);
assertEquals(Arrays.asList(connectorName1, connectorName2), new ArrayList<>(configState.connectors())); assertEquals(List.of(connectorName1, connectorName2), new ArrayList<>(configState.connectors()));
assertEquals(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName1)); assertEquals(List.of(TASK_IDS.get(0), TASK_IDS.get(1)), configState.tasks(connectorName1));
assertEquals(Collections.singletonList(TASK_IDS.get(2)), configState.tasks(connectorName2)); 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(0), configState.taskConfig(TASK_IDS.get(0)));
assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1))); assertEquals(SAMPLE_CONFIGS.get(1), configState.taskConfig(TASK_IDS.get(1)));
assertEquals(SAMPLE_CONFIGS.get(2), configState.taskConfig(TASK_IDS.get(2))); 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 // 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)));
verify(configUpdateListener).onTaskConfigUpdate(Collections.singletonList(TASK_IDS.get(2))); verify(configUpdateListener).onTaskConfigUpdate(List.of(TASK_IDS.get(2)));
configStorage.stop(); configStorage.stop();
verify(configLog).stop(); verify(configLog).stop();

View File

@ -45,9 +45,9 @@ import org.mockito.quality.Strictness;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
import java.util.Set; import java.util.Set;
@ -145,7 +145,7 @@ public class KafkaOffsetBackingStoreTest {
if (mockKeyConverter) { if (mockKeyConverter) {
when(keyConverter.toConnectData(any(), any())).thenReturn(new SchemaAndValue(null, 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)); store = spy(new KafkaOffsetBackingStore(adminSupplier, clientIdBase, keyConverter));
@ -233,7 +233,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting from empty store should return nulls // 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 // Since we didn't read them yet, these will be null
assertNull(offsets.get(TP0_KEY)); assertNull(offsets.get(TP0_KEY));
assertNull(offsets.get(TP1_KEY)); assertNull(offsets.get(TP1_KEY));
@ -270,7 +270,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it // 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(TP0_VALUE, offsets.get(TP0_KEY));
assertEquals(TP1_VALUE, offsets.get(TP1_KEY)); assertEquals(TP1_VALUE, offsets.get(TP1_KEY));
@ -287,7 +287,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it // 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(TP0_VALUE_NEW, offsets.get(TP0_KEY));
assertEquals(TP1_VALUE_NEW, offsets.get(TP1_KEY)); assertEquals(TP1_VALUE_NEW, offsets.get(TP1_KEY));
@ -363,7 +363,7 @@ public class KafkaOffsetBackingStoreTest {
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Getting data should read to end of our published data and return it // 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)); 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. // 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 @Test
public void testConnectorPartitions() throws Exception { public void testConnectorPartitions() throws Exception {
JsonConverter jsonConverter = new JsonConverter(); 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(() -> { store = spy(new KafkaOffsetBackingStore(() -> {
fail("Should not attempt to instantiate admin in these tests"); fail("Should not attempt to instantiate admin in these tests");
return null; return null;
@ -506,57 +506,57 @@ public class KafkaOffsetBackingStoreTest {
doAnswer(invocation -> { doAnswer(invocation -> {
capturedConsumedCallback.getValue().onCompletion(null, capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0, new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1", jsonConverter.fromConnectData("", null, List.of("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), TP0_VALUE.array(), Map.of("partitionKey", "partitionValue1"))), TP0_VALUE.array(),
new RecordHeaders(), Optional.empty())); new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null, capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0, new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1", jsonConverter.fromConnectData("", null, List.of("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), TP1_VALUE.array(), Map.of("partitionKey", "partitionValue1"))), TP1_VALUE.array(),
new RecordHeaders(), Optional.empty())); new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null, capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0, new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1", jsonConverter.fromConnectData("", null, List.of("connector1",
Collections.singletonMap("partitionKey", "partitionValue2"))), TP2_VALUE.array(), Map.of("partitionKey", "partitionValue2"))), TP2_VALUE.array(),
new RecordHeaders(), Optional.empty())); new RecordHeaders(), Optional.empty()));
capturedConsumedCallback.getValue().onCompletion(null, capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0, new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector2", jsonConverter.fromConnectData("", null, List.of("connector2",
Collections.singletonMap("partitionKey", "partitionValue"))), TP1_VALUE.array(), Map.of("partitionKey", "partitionValue"))), TP1_VALUE.array(),
new RecordHeaders(), Optional.empty())); new RecordHeaders(), Optional.empty()));
storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null); storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null);
return null; return null;
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Trigger a read to the end of the log // 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>> connectorPartitions1 = store.connectorPartitions("connector1");
Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>(); Set<Map<String, Object>> expectedConnectorPartition1 = new HashSet<>();
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue1")); expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue1"));
expectedConnectorPartition1.add(Collections.singletonMap("partitionKey", "partitionValue2")); expectedConnectorPartition1.add(Map.of("partitionKey", "partitionValue2"));
assertEquals(expectedConnectorPartition1, connectorPartitions1); assertEquals(expectedConnectorPartition1, connectorPartitions1);
Set<Map<String, Object>> connectorPartitions2 = store.connectorPartitions("connector2"); 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); assertEquals(expectedConnectorPartition2, connectorPartitions2);
doAnswer(invocation -> { doAnswer(invocation -> {
capturedConsumedCallback.getValue().onCompletion(null, capturedConsumedCallback.getValue().onCompletion(null,
new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0, new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0, 0,
jsonConverter.fromConnectData("", null, Arrays.asList("connector1", jsonConverter.fromConnectData("", null, List.of("connector1",
Collections.singletonMap("partitionKey", "partitionValue1"))), null, Map.of("partitionKey", "partitionValue1"))), null,
new RecordHeaders(), Optional.empty())); new RecordHeaders(), Optional.empty()));
storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null); storeLogCallbackArgumentCaptor.getValue().onCompletion(null, null);
return null; return null;
}).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture()); }).when(storeLog).readToEnd(storeLogCallbackArgumentCaptor.capture());
// Trigger a read to the end of the log // 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 // Null valued offset for a partition key should remove that partition for the connector
connectorPartitions1 = store.connectorPartitions("connector1"); connectorPartitions1 = store.connectorPartitions("connector1");
assertEquals(Collections.singleton(Collections.singletonMap("partitionKey", "partitionValue2")), connectorPartitions1); assertEquals(Set.of(Map.of("partitionKey", "partitionValue2")), connectorPartitions1);
store.stop(); store.stop();
verify(storeLog).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.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@ -75,7 +75,7 @@ public class KafkaStatusBackingStoreFormatTest {
public void setup() { public void setup() {
time = new MockTime(); time = new MockTime();
JsonConverter converter = new JsonConverter(); 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); store = new KafkaStatusBackingStore(new MockTime(), converter, STATUS_TOPIC, () -> null, kafkaBasedLog);
} }
@ -182,7 +182,7 @@ public class KafkaStatusBackingStoreFormatTest {
store.read(statusRecord); store.read(statusRecord);
assertTrue(store.topics.containsKey("bar")); assertTrue(store.topics.containsKey("bar"));
assertFalse(store.topics.get("bar").containsKey("foo")); assertFalse(store.topics.get("bar").containsKey("foo"));
assertEquals(Collections.emptyMap(), store.topics.get("bar")); assertEquals(Map.of(), store.topics.get("bar"));
} }
@Test @Test
@ -204,7 +204,7 @@ public class KafkaStatusBackingStoreFormatTest {
ConsumerRecord<String, byte[]> statusRecord = new ConsumerRecord<>(STATUS_TOPIC, 0, 0, key, valueCaptor.getValue()); ConsumerRecord<String, byte[]> statusRecord = new ConsumerRecord<>(STATUS_TOPIC, 0, 0, key, valueCaptor.getValue());
store.read(statusRecord); store.read(statusRecord);
assertEquals(topicStatus, store.getTopic(FOO_CONNECTOR, FOO_TOPIC)); 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 @Test

View File

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

View File

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

View File

@ -54,8 +54,8 @@ import static org.mockito.Mockito.when;
public class OffsetStorageWriterTest { public class OffsetStorageWriterTest {
private static final String NAMESPACE = "namespace"; private static final String NAMESPACE = "namespace";
// Connect format - any types should be accepted here // 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_KEY = Map.of("key", "key");
private static final Map<String, Object> OFFSET_VALUE = Collections.singletonMap("key", 12); private static final Map<String, Object> OFFSET_VALUE = Map.of("key", 12);
// Serialized // Serialized
private static final byte[] OFFSET_KEY_SERIALIZED = "key-serialized".getBytes(); 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.nio.charset.StandardCharsets;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -40,7 +40,7 @@ public class OffsetUtilsTest {
private static final JsonConverter CONVERTER = new JsonConverter(); private static final JsonConverter CONVERTER = new JsonConverter();
static { static {
CONVERTER.configure(Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true); CONVERTER.configure(Map.of(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"), true);
} }
@Test @Test
@ -60,18 +60,18 @@ public class OffsetUtilsTest {
@Test @Test
public void testValidateFormatMapWithNonPrimitiveKeys() { 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)); DataException e = assertThrows(DataException.class, () -> OffsetUtils.validateFormat(offsetData));
assertTrue(e.getMessage().contains("Offsets may only contain primitive types as values")); 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)); e = assertThrows(DataException.class, () -> OffsetUtils.validateFormat(offsetData2));
assertTrue(e.getMessage().contains("Offsets may only contain primitive types as values")); assertTrue(e.getMessage().contains("Offsets may only contain primitive types as values"));
} }
@Test @Test
public void testValidateFormatWithValidFormat() { 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 // Expect no exception to be thrown
OffsetUtils.validateFormat(offsetData); OffsetUtils.validateFormat(offsetData);
} }
@ -99,17 +99,17 @@ public class OffsetUtilsTest {
@Test @Test
public void testProcessPartitionKeyListWithOneElement() { public void testProcessPartitionKeyListWithOneElement() {
assertInvalidPartitionKey( assertInvalidPartitionKey(
serializePartitionKey(Collections.singletonList("")), serializePartitionKey(List.of("")),
"Ignoring offset partition key with an unexpected number of elements"); "Ignoring offset partition key with an unexpected number of elements");
} }
@Test @Test
public void testProcessPartitionKeyListWithElementsOfWrongType() { public void testProcessPartitionKeyListWithElementsOfWrongType() {
assertInvalidPartitionKey( 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"); "Ignoring offset partition key with an unexpected format for the first element in the partition key list");
assertInvalidPartitionKey( 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"); "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() { public void testProcessPartitionKeyValidList() {
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) { try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) {
Map<String, Set<Map<String, Object>>> connectorPartitions = new HashMap<>(); 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(1, connectorPartitions.size());
assertEquals(0, logCaptureAppender.getMessages().size()); assertEquals(0, logCaptureAppender.getMessages().size());
} }
@ -139,7 +139,7 @@ public class OffsetUtilsTest {
try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) { try (LogCaptureAppender logCaptureAppender = LogCaptureAppender.createAndRegister(OffsetUtils.class)) {
Map<String, Set<Map<String, Object>>> connectorPartitions = new HashMap<>(); Map<String, Set<Map<String, Object>>> connectorPartitions = new HashMap<>();
OffsetUtils.processPartitionKey(serializePartitionKey(Arrays.asList("connector-name", null)), new byte[0], CONVERTER, connectorPartitions); 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()); 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.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.Optional; import java.util.Optional;
@ -125,7 +124,7 @@ public class ConnectUtilsTest {
"thanks to newly-introduced federal legislation", "thanks to newly-introduced federal legislation",
false) false)
); );
assertEquals(Collections.singletonMap("\u1984", "big brother"), props); assertEquals(Map.of("\u1984", "big brother"), props);
props.clear(); props.clear();
props.put("\u1984", "BIG BROTHER"); props.put("\u1984", "BIG BROTHER");
@ -141,7 +140,7 @@ public class ConnectUtilsTest {
"thanks to newly-introduced federal legislation", "thanks to newly-introduced federal legislation",
true) true)
); );
assertEquals(Collections.singletonMap("\u1984", "big brother"), props); assertEquals(Map.of("\u1984", "big brother"), props);
} }
@Test @Test

View File

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

View File

@ -26,7 +26,6 @@ import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness; import org.mockito.quality.Strictness;
import java.time.Duration; import java.time.Duration;
import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.function.Function; import java.util.function.Function;
@ -43,7 +42,7 @@ import static org.mockito.Mockito.when;
@MockitoSettings(strictness = Strictness.STRICT_STUBS) @MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class SharedTopicAdminTest { 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 TopicAdmin mockTopicAdmin;
@Mock private Function<Map<String, Object>, TopicAdmin> factory; @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 org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
@ -44,7 +43,7 @@ public class SinkUtilsTest {
connectorOffsets = SinkUtils.consumerGroupOffsetsToConnectorOffsets(consumerGroupOffsets); connectorOffsets = SinkUtils.consumerGroupOffsetsToConnectorOffsets(consumerGroupOffsets);
assertEquals(1, connectorOffsets.offsets().size()); 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<>(); Map<String, Object> expectedPartition = new HashMap<>();
expectedPartition.put(SinkUtils.KAFKA_TOPIC_KEY, "test-topic"); expectedPartition.put(SinkUtils.KAFKA_TOPIC_KEY, "test-topic");

View File

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

View File

@ -69,7 +69,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
import static org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; 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(); NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build();
Cluster cluster = createCluster(1); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, "myTopic", Collections.singletonList(topicPartitionInfo), null); mockAdminClient.addTopic(false, "myTopic", List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
assertFalse(admin.createTopic(newTopic)); assertFalse(admin.createTopic(newTopic));
assertTrue(admin.createTopics(newTopic).isEmpty()); assertTrue(admin.createTopics(newTopic).isEmpty());
@ -307,12 +306,12 @@ public class TopicAdminTest {
NewTopic newTopic = TopicAdmin.defineTopic(topicName).partitions(1).compacted().build(); NewTopic newTopic = TopicAdmin.defineTopic(topicName).partitions(1).compacted().build();
Cluster cluster = createCluster(1); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
Map<String, TopicDescription> desc = admin.describeTopics(newTopic.name()); Map<String, TopicDescription> desc = admin.describeTopics(newTopic.name());
assertFalse(desc.isEmpty()); 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); assertEquals(desc.get("myTopic"), topicDesc);
} }
} }
@ -381,14 +380,14 @@ public class TopicAdminTest {
public void describeTopicConfigShouldReturnTopicConfigWhenTopicExists() { public void describeTopicConfigShouldReturnTopicConfigWhenTopicExists() {
String topicName = "myTopic"; String topicName = "myTopic";
NewTopic newTopic = TopicAdmin.defineTopic(topicName) NewTopic newTopic = TopicAdmin.defineTopic(topicName)
.config(Collections.singletonMap("foo", "bar")) .config(Map.of("foo", "bar"))
.partitions(1) .partitions(1)
.compacted() .compacted()
.build(); .build();
Cluster cluster = createCluster(1); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), null);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
Map<String, Config> result = admin.describeTopicConfigs(newTopic.name()); Map<String, Config> result = admin.describeTopicConfigs(newTopic.name());
assertFalse(result.isEmpty()); assertFalse(result.isEmpty());
@ -438,11 +437,11 @@ public class TopicAdminTest {
@Test @Test
public void verifyingTopicCleanupPolicyShouldReturnTrueWhenTopicHasCorrectPolicy() { public void verifyingTopicCleanupPolicyShouldReturnTrueWhenTopicHasCorrectPolicy() {
String topicName = "myTopic"; 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); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
boolean result = admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"); boolean result = admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose");
assertTrue(result); assertTrue(result);
@ -452,11 +451,11 @@ public class TopicAdminTest {
@Test @Test
public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeletePolicy() { public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeletePolicy() {
String topicName = "myTopic"; 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); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose")); ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
assertTrue(e.getMessage().contains("to guarantee consistency and durability")); assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
@ -466,11 +465,11 @@ public class TopicAdminTest {
@Test @Test
public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeleteAndCompactPolicy() { public void verifyingTopicCleanupPolicyShouldFailWhenTopicHasDeleteAndCompactPolicy() {
String topicName = "myTopic"; 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); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose")); ConfigException e = assertThrows(ConfigException.class, () -> admin.verifyTopicCleanupPolicyOnlyCompact("myTopic", "worker.topic", "purpose"));
assertTrue(e.getMessage().contains("to guarantee consistency and durability")); assertTrue(e.getMessage().contains("to guarantee consistency and durability"));
@ -480,11 +479,11 @@ public class TopicAdminTest {
@Test @Test
public void verifyingGettingTopicCleanupPolicies() { public void verifyingGettingTopicCleanupPolicies() {
String topicName = "myTopic"; 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); Cluster cluster = createCluster(1);
try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) {
TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of());
mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), topicConfigs); mockAdminClient.addTopic(false, topicName, List.of(topicPartitionInfo), topicConfigs);
TopicAdmin admin = new TopicAdmin(mockAdminClient); TopicAdmin admin = new TopicAdmin(mockAdminClient);
Set<String> policies = admin.topicCleanupPolicy("myTopic"); Set<String> policies = admin.topicCleanupPolicy("myTopic");
assertEquals(1, policies.size()); assertEquals(1, policies.size());
@ -501,7 +500,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldRethrowUnknownVersionException() { public void retryEndOffsetsShouldRethrowUnknownVersionException() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); 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 Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -519,7 +518,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldWrapNonRetriableExceptionsWithConnectException() { public void retryEndOffsetsShouldWrapNonRetriableExceptionsWithConnectException() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1); Set<TopicPartition> tps = Set.of(tp1);
Long offset = 1000L; Long offset = 1000L;
Cluster cluster = createCluster(1, "myTopic", 1); Cluster cluster = createCluster(1, "myTopic", 1);
@ -548,7 +547,7 @@ public class TopicAdminTest {
public void retryEndOffsetsShouldRetryWhenTopicNotFound() { public void retryEndOffsetsShouldRetryWhenTopicNotFound() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1); Set<TopicPartition> tps = Set.of(tp1);
Long offset = 1000L; Long offset = 1000L;
Cluster cluster = createCluster(1, "myTopic", 1); Cluster cluster = createCluster(1, "myTopic", 1);
@ -560,7 +559,7 @@ public class TopicAdminTest {
TopicAdmin admin = new TopicAdmin(env.adminClient()); TopicAdmin admin = new TopicAdmin(env.adminClient());
Map<TopicPartition, Long> endoffsets = admin.retryEndOffsets(tps, Duration.ofMillis(100), 1); 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() { public void endOffsetsShouldFailWithNonRetriableWhenAuthorizationFailureOccurs() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); 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 Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -585,7 +584,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithUnsupportedVersionWhenVersionUnsupportedErrorOccurs() { public void endOffsetsShouldFailWithUnsupportedVersionWhenVersionUnsupportedErrorOccurs() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); 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 Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -601,7 +600,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithTimeoutExceptionWhenTimeoutErrorOccurs() { public void endOffsetsShouldFailWithTimeoutExceptionWhenTimeoutErrorOccurs() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); 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 Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv( try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(
@ -619,7 +618,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWithNonRetriableWhenUnknownErrorOccurs() { public void endOffsetsShouldFailWithNonRetriableWhenUnknownErrorOccurs() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); 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 Long offset = null; // response should use error
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -638,7 +637,7 @@ public class TopicAdminTest {
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
TopicAdmin admin = new TopicAdmin(env.adminClient()); 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()); assertTrue(offsets.isEmpty());
} }
} }
@ -647,7 +646,7 @@ public class TopicAdminTest {
public void endOffsetsShouldReturnOffsetsForOnePartition() { public void endOffsetsShouldReturnOffsetsForOnePartition() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1); Set<TopicPartition> tps = Set.of(tp1);
long offset = 1000L; long offset = 1000L;
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
@ -686,7 +685,7 @@ public class TopicAdminTest {
public void endOffsetsShouldFailWhenAnyTopicPartitionHasError() { public void endOffsetsShouldFailWhenAnyTopicPartitionHasError() {
String topicName = "myTopic"; String topicName = "myTopic";
TopicPartition tp1 = new TopicPartition(topicName, 0); TopicPartition tp1 = new TopicPartition(topicName, 0);
Set<TopicPartition> tps = Collections.singleton(tp1); Set<TopicPartition> tps = Set.of(tp1);
Cluster cluster = createCluster(1, topicName, 1); Cluster cluster = createCluster(1, topicName, 1);
try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) {
env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); env.kafkaClient().setNodeApiVersions(NodeApiVersions.create());
@ -718,8 +717,8 @@ public class TopicAdminTest {
"mockClusterId", "mockClusterId",
nodes.values(), nodes.values(),
pInfos, pInfos,
Collections.emptySet(), Set.of(),
Collections.emptySet(), Set.of(),
leader); leader);
} }
@ -737,9 +736,9 @@ public class TopicAdminTest {
.setPartitionIndex(pInfo.partition()) .setPartitionIndex(pInfo.partition())
.setLeaderId(pInfo.leader().id()) .setLeaderId(pInfo.leader().id())
.setLeaderEpoch(234) .setLeaderEpoch(234)
.setReplicaNodes(Arrays.stream(pInfo.replicas()).map(Node::id).collect(Collectors.toList())) .setReplicaNodes(Arrays.stream(pInfo.replicas()).map(Node::id).toList())
.setIsrNodes(Arrays.stream(pInfo.inSyncReplicas()).map(Node::id).collect(Collectors.toList())) .setIsrNodes(Arrays.stream(pInfo.inSyncReplicas()).map(Node::id).toList())
.setOfflineReplicas(Arrays.stream(pInfo.offlineReplicas()).map(Node::id).collect(Collectors.toList())); .setOfflineReplicas(Arrays.stream(pInfo.offlineReplicas()).map(Node::id).toList());
pms.add(pm); pms.add(pm);
} }
MetadataResponseTopic tm = new MetadataResponseTopic() MetadataResponseTopic tm = new MetadataResponseTopic()
@ -787,7 +786,7 @@ public class TopicAdminTest {
} }
private ListOffsetsResponse listOffsetsResult(TopicPartition tp1, Long offset1) { 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) { 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) protected TopicDescription topicDescription(MockAdminClient admin, String topicName)
throws ExecutionException, InterruptedException { throws ExecutionException, InterruptedException {
DescribeTopicsResult result = admin.describeTopics(Collections.singleton(topicName)); DescribeTopicsResult result = admin.describeTopics(Set.of(topicName));
Map<String, KafkaFuture<TopicDescription>> byName = result.topicNameValues(); Map<String, KafkaFuture<TopicDescription>> byName = result.topicNameValues();
return byName.get(topicName).get(); return byName.get(topicName).get();
} }
@ -958,8 +957,8 @@ public class TopicAdminTest {
.map(e -> new DescribeConfigsResponseData.DescribeConfigsResourceResult() .map(e -> new DescribeConfigsResponseData.DescribeConfigsResourceResult()
.setName(e.getKey()) .setName(e.getKey())
.setValue(e.getValue())) .setValue(e.getValue()))
.collect(Collectors.toList()))) .toList()))
.collect(Collectors.toList()); .toList();
return new DescribeConfigsResponse(new DescribeConfigsResponseData().setThrottleTimeMs(1000).setResults(results)); 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.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -154,7 +153,7 @@ public class TopicCreationTest {
assertFalse(topicCreation.isTopicCreationEnabled()); assertFalse(topicCreation.isTopicCreationEnabled());
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertNull(topicCreation.defaultTopicGroup()); assertNull(topicCreation.defaultTopicGroup());
assertEquals(Collections.emptyMap(), topicCreation.topicGroups()); assertEquals(Map.of(), topicCreation.topicGroups());
assertNull(topicCreation.findFirstGroup(FOO_TOPIC)); assertNull(topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -169,7 +168,7 @@ public class TopicCreationTest {
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertNull(topicCreation.defaultTopicGroup()); assertNull(topicCreation.defaultTopicGroup());
assertEquals(0, topicCreation.topicGroups().size()); assertEquals(0, topicCreation.topicGroups().size());
assertEquals(Collections.emptyMap(), topicCreation.topicGroups()); assertEquals(Map.of(), topicCreation.topicGroups());
assertNull(topicCreation.findFirstGroup(FOO_TOPIC)); assertNull(topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -187,14 +186,14 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(DEFAULT_PARTITIONS, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_PARTITIONS, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(1, groups.size()); 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 // verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -205,7 +204,7 @@ public class TopicCreationTest {
assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name()); assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name());
assertTrue(topicCreation.isTopicCreationEnabled()); assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(Collections.emptyMap(), topicCreation.topicGroups()); assertEquals(Map.of(), topicCreation.topicGroups());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC)); assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -215,7 +214,7 @@ public class TopicCreationTest {
assertEquals(FOO_TOPIC, topicSpec.name()); assertEquals(FOO_TOPIC, topicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, topicSpec.replicationFactor()); assertEquals(DEFAULT_REPLICATION_FACTOR, topicSpec.replicationFactor());
assertEquals(DEFAULT_PARTITIONS, topicSpec.numPartitions()); assertEquals(DEFAULT_PARTITIONS, topicSpec.numPartitions());
assertEquals(Collections.emptyMap(), topicSpec.configs()); assertEquals(Map.of(), topicSpec.configs());
} }
@Test @Test
@ -241,14 +240,14 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(replicas, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(replicas, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(topicProps, sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(topicProps, sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
assertEquals(1, groups.size()); 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 // verify topic creation
TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups);
@ -259,7 +258,7 @@ public class TopicCreationTest {
assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name()); assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name());
assertTrue(topicCreation.isTopicCreationEnabled()); assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(Collections.emptyMap(), topicCreation.topicGroups()); assertEquals(Map.of(), topicCreation.topicGroups());
assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC)); assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -292,9 +291,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -318,7 +317,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationEnabled()); assertTrue(topicCreation.isTopicCreationEnabled());
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertEquals(1, topicCreation.topicGroups().size()); 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(FOO_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
@ -328,7 +327,7 @@ public class TopicCreationTest {
assertEquals(BAR_TOPIC, defaultTopicSpec.name()); assertEquals(BAR_TOPIC, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor()); assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions()); assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs()); assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name()); assertEquals(FOO_TOPIC, fooTopicSpec.name());
@ -357,9 +356,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -384,7 +383,7 @@ public class TopicCreationTest {
assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC));
assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC)); assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC));
assertEquals(1, topicCreation.topicGroups().size()); 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(FOO_TOPIC));
assertEquals(fooGroup, topicCreation.findFirstGroup(BAR_TOPIC)); assertEquals(fooGroup, topicCreation.findFirstGroup(BAR_TOPIC));
topicCreation.addTopic(FOO_TOPIC); topicCreation.addTopic(FOO_TOPIC);
@ -434,9 +433,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -479,7 +478,7 @@ public class TopicCreationTest {
assertEquals(otherTopic, defaultTopicSpec.name()); assertEquals(otherTopic, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor()); assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions()); assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs()); assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name()); assertEquals(FOO_TOPIC, fooTopicSpec.name());
@ -566,9 +565,9 @@ public class TopicCreationTest {
assertTrue(sourceConfig.usesTopicCreation()); assertTrue(sourceConfig.usesTopicCreation());
assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.singletonList(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(".*"), sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyList(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(List.of(), sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP));
assertEquals(Collections.emptyMap(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP)); assertEquals(Map.of(), sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP));
// verify topic creation group is instantiated correctly // verify topic creation group is instantiated correctly
Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig); Map<String, TopicCreationGroup> groups = TopicCreationGroup.configuredGroups(sourceConfig);
@ -611,7 +610,7 @@ public class TopicCreationTest {
assertEquals(otherTopic, defaultTopicSpec.name()); assertEquals(otherTopic, defaultTopicSpec.name());
assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor()); assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor());
assertEquals(partitions, defaultTopicSpec.numPartitions()); assertEquals(partitions, defaultTopicSpec.numPartitions());
assertEquals(Collections.emptyMap(), defaultTopicSpec.configs()); assertEquals(Map.of(), defaultTopicSpec.configs());
NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC);
assertEquals(FOO_TOPIC, fooTopicSpec.name()); assertEquals(FOO_TOPIC, fooTopicSpec.name());

View File

@ -47,7 +47,6 @@ import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Collection; import java.util.Collection;
import java.util.Collections;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -282,7 +281,7 @@ abstract class EmbeddedConnect {
throw new ConnectException("Failed to serialize connector creation request: " + createConnectorRequest); 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()) { if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) {
return responseToString(response); return responseToString(response);
} else { } else {
@ -449,7 +448,7 @@ abstract class EmbeddedConnect {
*/ */
public void restartConnector(String connName) { public void restartConnector(String connName) {
String url = endpointForResource(String.format("connectors/%s/restart", 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()) { if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) {
throw new ConnectRestException(response.getStatus(), throw new ConnectRestException(response.getStatus(),
"Could not execute POST request. Error response: " + responseToString(response)); "Could not execute POST request. Error response: " + responseToString(response));
@ -466,7 +465,7 @@ abstract class EmbeddedConnect {
*/ */
public void restartTask(String connName, int taskNum) { public void restartTask(String connName, int taskNum) {
String url = endpointForResource(String.format("connectors/%s/tasks/%d/restart", connName, 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()) { if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) {
throw new ConnectRestException(response.getStatus(), throw new ConnectRestException(response.getStatus(),
"Could not execute POST request. Error response: " + responseToString(response)); "Could not execute POST request. Error response: " + responseToString(response));
@ -492,7 +491,7 @@ abstract class EmbeddedConnect {
} else { } else {
restartEndpoint = endpointForResource(restartPath); restartEndpoint = endpointForResource(restartPath);
} }
Response response = requestPost(restartEndpoint, "", Collections.emptyMap()); Response response = requestPost(restartEndpoint, "", Map.of());
try { try {
if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) {
//only the 202 status returns a body //only the 202 status returns a body
@ -577,7 +576,7 @@ abstract class EmbeddedConnect {
.readerFor(new TypeReference<Map<String, Map<String, List<String>>>>() { }) .readerFor(new TypeReference<Map<String, Map<String, List<String>>>>() { })
.readValue(responseToString(response)); .readValue(responseToString(response));
return new ActiveTopicsInfo(connectorName, return new ActiveTopicsInfo(connectorName,
activeTopics.get(connectorName).getOrDefault("topics", Collections.emptyList())); activeTopics.get(connectorName).getOrDefault("topics", List.of()));
} }
} catch (IOException e) { } catch (IOException e) {
log.error("Could not read connector state from response: {}", 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) { public String alterSourceConnectorOffset(String connectorName, Map<String, ?> partition, Map<String, ?> offset) {
return alterConnectorOffsets( return alterConnectorOffsets(
connectorName, 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) { public String alterSinkConnectorOffset(String connectorName, TopicPartition topicPartition, Long offset) {
return alterConnectorOffsets( return alterConnectorOffsets(
connectorName, 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 * @throws ConnectException if execution of the GET request fails
*/ */
public Response requestGet(String url) { 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 * @throws ConnectException if execution of the PUT request fails
*/ */
public Response requestPut(String url, String body) { 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 * @throws ConnectException if execution of the PATCH request fails
*/ */
public Response requestPatch(String url, String body) { 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 * @throws ConnectException if execution of the DELETE request fails
*/ */
public Response requestDelete(String url) { 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.io.UncheckedIOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; import java.util.Objects;
@ -109,8 +108,8 @@ public class EmbeddedConnectStandalone extends EmbeddedConnect {
@Override @Override
protected Set<WorkerHandle> workers() { protected Set<WorkerHandle> workers() {
return connectWorker != null return connectWorker != null
? Collections.singleton(connectWorker) ? Set.of(connectWorker)
: Collections.emptySet(); : Set.of();
} }
public Response healthCheck() { public Response healthCheck() {

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -73,14 +73,13 @@ public class AclCommand {
public static void main(String[] args) { public static void main(String[] args) {
AclCommandOptions opts = new AclCommandOptions(args); AclCommandOptions opts = new AclCommandOptions(args);
AdminClientService aclCommandService = new AdminClientService(opts);
try (Admin admin = Admin.create(adminConfigs(opts))) { try (Admin admin = Admin.create(adminConfigs(opts))) {
if (opts.options.has(opts.addOpt)) { if (opts.options.has(opts.addOpt)) {
aclCommandService.addAcls(admin); addAcls(admin, opts);
} else if (opts.options.has(opts.removeOpt)) { } else if (opts.options.has(opts.removeOpt)) {
aclCommandService.removeAcls(admin); removeAcls(admin, opts);
} else if (opts.options.has(opts.listOpt)) { } else if (opts.options.has(opts.listOpt)) {
aclCommandService.listAcls(admin); listAcls(admin, opts);
} }
} catch (Throwable e) { } catch (Throwable e) {
System.out.println("Error while executing ACL command: " + e.getMessage()); System.out.println("Error while executing ACL command: " + e.getMessage());
@ -102,108 +101,99 @@ public class AclCommand {
return props; return props;
} }
private static class AdminClientService { private static void addAcls(Admin admin, AclCommandOptions opts) throws ExecutionException, InterruptedException {
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcl = getResourceToAcls(opts);
private final AclCommandOptions opts; for (Map.Entry<ResourcePattern, Set<AccessControlEntry>> entry : resourceToAcl.entrySet()) {
ResourcePattern resource = entry.getKey();
AdminClientService(AclCommandOptions opts) { Set<AccessControlEntry> acls = entry.getValue();
this.opts = opts; System.out.println("Adding ACLs for resource `" + resource + "`: " + NL + " " + acls.stream().map(a -> "\t" + a).collect(Collectors.joining(NL)) + NL);
Collection<AclBinding> aclBindings = acls.stream().map(acl -> new AclBinding(resource, acl)).collect(Collectors.toList());
admin.createAcls(aclBindings).all().get();
} }
}
void addAcls(Admin admin) throws ExecutionException, InterruptedException { private static void removeAcls(Admin admin, AclCommandOptions opts) throws ExecutionException, InterruptedException {
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcl = getResourceToAcls(opts); Map<ResourcePatternFilter, Set<AccessControlEntry>> filterToAcl = getResourceFilterToAcls(opts);
for (Map.Entry<ResourcePattern, Set<AccessControlEntry>> entry : resourceToAcl.entrySet()) { for (Map.Entry<ResourcePatternFilter, Set<AccessControlEntry>> entry : filterToAcl.entrySet()) {
ResourcePattern resource = entry.getKey(); ResourcePatternFilter filter = entry.getKey();
Set<AccessControlEntry> acls = entry.getValue(); Set<AccessControlEntry> acls = entry.getValue();
System.out.println("Adding ACLs for resource `" + resource + "`: " + NL + " " + acls.stream().map(a -> "\t" + a).collect(Collectors.joining(NL)) + NL);
Collection<AclBinding> aclBindings = acls.stream().map(acl -> new AclBinding(resource, acl)).collect(Collectors.toList());
admin.createAcls(aclBindings).all().get();
}
}
void removeAcls(Admin admin) throws ExecutionException, InterruptedException {
Map<ResourcePatternFilter, Set<AccessControlEntry>> filterToAcl = getResourceFilterToAcls(opts);
for (Map.Entry<ResourcePatternFilter, Set<AccessControlEntry>> entry : filterToAcl.entrySet()) {
ResourcePatternFilter filter = entry.getKey();
Set<AccessControlEntry> acls = entry.getValue();
if (acls.isEmpty()) {
if (confirmAction(opts, "Are you sure you want to delete all ACLs for resource filter `" + filter + "`? (y/n)")) {
removeAcls(admin, acls, filter);
}
} else {
String msg = "Are you sure you want to remove ACLs: " + NL +
" " + acls.stream().map(a -> "\t" + a).collect(Collectors.joining(NL)) + NL +
" from resource filter `" + filter + "`? (y/n)";
if (confirmAction(opts, msg)) {
removeAcls(admin, acls, filter);
}
}
}
}
private void listAcls(Admin admin) throws ExecutionException, InterruptedException {
Set<ResourcePatternFilter> filters = getResourceFilter(opts, false);
Set<KafkaPrincipal> listPrincipals = getPrincipals(opts, opts.listPrincipalsOpt);
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls = getAcls(admin, filters);
if (listPrincipals.isEmpty()) {
printResourceAcls(resourceToAcls);
} else {
listPrincipals.forEach(principal -> {
System.out.println("ACLs for principal `" + principal + "`");
Map<ResourcePattern, Set<AccessControlEntry>> filteredResourceToAcls = resourceToAcls.entrySet().stream()
.map(entry -> {
ResourcePattern resource = entry.getKey();
Set<AccessControlEntry> acls = entry.getValue().stream()
.filter(acl -> principal.toString().equals(acl.principal()))
.collect(Collectors.toSet());
return new AbstractMap.SimpleEntry<>(resource, acls);
})
.filter(entry -> !entry.getValue().isEmpty())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
printResourceAcls(filteredResourceToAcls);
});
}
}
private static void printResourceAcls(Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls) {
resourceToAcls.forEach((resource, acls) ->
System.out.println("Current ACLs for resource `" + resource + "`:" + NL +
acls.stream().map(acl -> "\t" + acl).collect(Collectors.joining(NL)) + NL)
);
}
private static void removeAcls(Admin adminClient, Set<AccessControlEntry> acls, ResourcePatternFilter filter) throws ExecutionException, InterruptedException {
if (acls.isEmpty()) { if (acls.isEmpty()) {
adminClient.deleteAcls(List.of(new AclBindingFilter(filter, AccessControlEntryFilter.ANY))).all().get(); if (confirmAction(opts, "Are you sure you want to delete all ACLs for resource filter `" + filter + "`? (y/n)")) {
removeAcls(admin, acls, filter);
}
} else { } else {
List<AclBindingFilter> aclBindingFilters = acls.stream().map(acl -> new AclBindingFilter(filter, acl.toFilter())).collect(Collectors.toList()); String msg = "Are you sure you want to remove ACLs: " + NL +
adminClient.deleteAcls(aclBindingFilters).all().get(); " " + acls.stream().map(a -> "\t" + a).collect(Collectors.joining(NL)) + NL +
} " from resource filter `" + filter + "`? (y/n)";
} if (confirmAction(opts, msg)) {
removeAcls(admin, acls, filter);
private 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();
} else {
aclBindings = new ArrayList<>();
for (ResourcePatternFilter filter : filters) {
aclBindings.addAll(adminClient.describeAcls(new AclBindingFilter(filter, AccessControlEntryFilter.ANY)).values().get());
} }
} }
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls = new HashMap<>();
for (AclBinding aclBinding : aclBindings) {
ResourcePattern resource = aclBinding.pattern();
Set<AccessControlEntry> acls = resourceToAcls.getOrDefault(resource, new HashSet<>());
acls.add(aclBinding.entry());
resourceToAcls.put(resource, acls);
}
return resourceToAcls;
} }
} }
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);
if (listPrincipals.isEmpty()) {
printResourceAcls(resourceToAcls);
} else {
listPrincipals.forEach(principal -> {
System.out.println("ACLs for principal `" + principal + "`");
Map<ResourcePattern, Set<AccessControlEntry>> filteredResourceToAcls = resourceToAcls.entrySet().stream()
.map(entry -> {
ResourcePattern resource = entry.getKey();
Set<AccessControlEntry> acls = entry.getValue().stream()
.filter(acl -> principal.toString().equals(acl.principal()))
.collect(Collectors.toSet());
return new AbstractMap.SimpleEntry<>(resource, acls);
})
.filter(entry -> !entry.getValue().isEmpty())
.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
printResourceAcls(filteredResourceToAcls);
});
}
}
private static void printResourceAcls(Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls) {
resourceToAcls.forEach((resource, acls) ->
System.out.println("Current ACLs for resource `" + resource + "`:" + NL +
acls.stream().map(acl -> "\t" + acl).collect(Collectors.joining(NL)) + NL)
);
}
private static void removeAcls(Admin adminClient, Set<AccessControlEntry> acls, ResourcePatternFilter filter) throws ExecutionException, InterruptedException {
if (acls.isEmpty()) {
adminClient.deleteAcls(List.of(new AclBindingFilter(filter, AccessControlEntryFilter.ANY))).all().get();
} else {
List<AclBindingFilter> aclBindingFilters = acls.stream().map(acl -> new AclBindingFilter(filter, acl.toFilter())).collect(Collectors.toList());
adminClient.deleteAcls(aclBindingFilters).all().get();
}
}
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();
} else {
aclBindings = new ArrayList<>();
for (ResourcePatternFilter filter : filters) {
aclBindings.addAll(adminClient.describeAcls(new AclBindingFilter(filter, AccessControlEntryFilter.ANY)).values().get());
}
}
Map<ResourcePattern, Set<AccessControlEntry>> resourceToAcls = new HashMap<>();
for (AclBinding aclBinding : aclBindings) {
ResourcePattern resource = aclBinding.pattern();
Set<AccessControlEntry> acls = resourceToAcls.getOrDefault(resource, new HashSet<>());
acls.add(aclBinding.entry());
resourceToAcls.put(resource, acls);
}
return resourceToAcls;
}
private static Map<ResourcePattern, Set<AccessControlEntry>> getResourceToAcls(AclCommandOptions opts) { private static Map<ResourcePattern, Set<AccessControlEntry>> getResourceToAcls(AclCommandOptions opts) {
PatternType patternType = opts.options.valueOf(opts.resourcePatternType); PatternType patternType = opts.options.valueOf(opts.resourcePatternType);
if (!patternType.isSpecific()) { if (!patternType.isSpecific()) {

View File

@ -196,22 +196,8 @@ public class ConnectPluginPath {
LIST, SYNC_MANIFESTS LIST, SYNC_MANIFESTS
} }
private static class Config { private record Config(Command command, Set<Path> locations, boolean dryRun, boolean keepNotFound, PrintStream out,
private final Command command; PrintStream err) {
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;
}
@Override @Override
public String toString() { 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 * <p>This is unique to the (source, class, type) tuple, and contains additional pre-computed information
* that pertains to this specific plugin. * that pertains to this specific plugin.
*/ */
private static class Row { private record Row(ManifestWorkspace.SourceWorkspace<?> workspace, String className, PluginType type,
private final ManifestWorkspace.SourceWorkspace<?> workspace; String version, List<String> aliases, boolean loadable, boolean hasManifest) {
private final String className; private Row(ManifestWorkspace.SourceWorkspace<?> workspace, String className, PluginType type, String version, List<String> aliases, boolean loadable, boolean hasManifest) {
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) {
this.workspace = Objects.requireNonNull(workspace, "workspace must be non-null"); this.workspace = Objects.requireNonNull(workspace, "workspace must be non-null");
this.className = Objects.requireNonNull(className, "className must be non-null"); this.className = Objects.requireNonNull(className, "className must be non-null");
this.version = Objects.requireNonNull(version, "version must be non-null"); this.version = Objects.requireNonNull(version, "version must be non-null");
@ -281,10 +260,6 @@ public class ConnectPluginPath {
this.hasManifest = hasManifest; this.hasManifest = hasManifest;
} }
private boolean loadable() {
return loadable;
}
private boolean compatible() { private boolean compatible() {
return loadable && hasManifest; 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.format;
import static java.lang.String.valueOf; import static java.lang.String.valueOf;
import static java.util.Arrays.asList;
/** /**
* A tool for describing quorum status * A tool for describing quorum status
@ -206,7 +205,7 @@ public class MetadataQuorumCommand {
rows.addAll(quorumInfoToRows(leader, quorumInfo.observers().stream(), "Observer", humanReadable)); rows.addAll(quorumInfoToRows(leader, quorumInfo.observers().stream(), "Observer", humanReadable));
ToolsUtils.prettyPrintTable( ToolsUtils.prettyPrintTable(
asList("NodeId", "DirectoryId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"), List.of("NodeId", "DirectoryId", "LogEndOffset", "Lag", "LastFetchTimestamp", "LastCaughtUpTimestamp", "Status"),
rows, rows,
System.out System.out
); );

View File

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

View File

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

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